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 2016/02/15 08:51:45 UTC

[01/50] [abbrv] ignite git commit: ignite-split2 - more fixes

Repository: ignite
Updated Branches:
  refs/heads/ignite-1232 [created] 76d5635f3


ignite-split2 - more fixes


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

Branch: refs/heads/ignite-1232
Commit: 4e16d0d8abbec6b824d8202144fee3f1175656c3
Parents: f90617b
Author: S.Vladykin <sv...@gridgain.com>
Authored: Wed Dec 2 02:44:00 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Wed Dec 2 02:44:00 2015 +0300

----------------------------------------------------------------------
 .../processors/query/h2/IgniteH2Indexing.java   | 15 +++-
 .../query/h2/opt/GridH2IndexBase.java           | 77 ++++++++++++++++----
 .../query/h2/opt/GridH2QueryContext.java        |  9 ---
 .../processors/query/h2/opt/GridH2Table.java    |  2 +-
 .../query/h2/opt/GridH2TreeIndex.java           |  2 +-
 .../query/h2/sql/GridSqlQueryParser.java        | 18 ++++-
 .../query/h2/sql/GridSqlQuerySplitter.java      | 31 +++++---
 7 files changed, 117 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4e16d0d8/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index f370f62..9424bfc 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -121,9 +121,11 @@ import org.apache.ignite.spi.indexing.IndexingQueryFilter;
 import org.h2.api.JavaObjectSerializer;
 import org.h2.command.CommandInterface;
 import org.h2.command.dml.OptimizerHints;
+import org.h2.engine.Session;
 import org.h2.engine.SysProperties;
 import org.h2.index.Index;
 import org.h2.index.SpatialIndex;
+import org.h2.jdbc.JdbcConnection;
 import org.h2.jdbc.JdbcPreparedStatement;
 import org.h2.message.DbException;
 import org.h2.mvstore.cache.CacheLongKeyLIRS;
@@ -1026,6 +1028,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         return !cctx.isReplicated() && !cctx.isLocal();
     }
 
+    /**
+     * @param c Connection.
+     * @return Session.
+     */
+    public static Session session(Connection c) {
+        return (Session)((JdbcConnection)c).getSession();
+    }
+
     /** {@inheritDoc} */
     @Override public QueryCursor<List<?>> queryTwoStep(GridCacheContext<?,?> cctx, SqlFieldsQuery qry) {
         final String space = cctx.name();
@@ -1046,10 +1056,12 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
         else {
             final UUID locNodeId = ctx.localNodeId();
+            Session ses = session(c);
 
-            enforceJoinOrder(enforceJoinOrder);
             GridH2QueryContext.set(new GridH2QueryContext(locNodeId, locNodeId, 0, PREPARE)
                 .distributedJoins(isPartitioned(cctx)));
+            enforceJoinOrder(enforceJoinOrder);
+            ses.setJoinBatchEnabled(false);
 
             PreparedStatement stmt;
 
@@ -1060,6 +1072,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 throw new CacheException("Failed to parse query: " + sqlQry, e);
             }
             finally {
+                ses.setJoinBatchEnabled(true);
                 enforceJoinOrder(false);
                 GridH2QueryContext.clear(false);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e16d0d8/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
index 154377b..c1666fd 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.query.h2.opt;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicLong;
@@ -138,14 +139,41 @@ public abstract class GridH2IndexBase extends BaseIndex {
     }
 
     /**
-     * @param states States map.
+     * @param qry Query.
      * @return {@code true} If it was proved that the query is fully collocated.
      */
-    public static boolean tryProveCollocated(Map<TableFilter,GridH2TableFilterCollocation> states) {
-        // We can't use values from the state cache because they can be outdated, but the cache
-        // must contain all the filters by the end of optimization.
-        // TODO
-        return false;
+    public static boolean isCollocated(Query qry) {
+        if (qry.isUnion()) {
+            SelectUnion union = (SelectUnion)qry;
+
+            return isCollocated(union.getLeft()) && isCollocated(union.getRight());
+        }
+
+        Select select = (Select)qry;
+
+        ArrayList<TableFilter> list = new ArrayList<>();
+
+        TableFilter f = select.getTopTableFilter();
+
+        assert f != null;
+
+        do {
+            list.add(f);
+
+            f = f.getJoin();
+        }
+        while (f != null);
+
+        TableFilter[] filters = list.toArray(new TableFilter[list.size()]);
+
+        Map<TableFilter,GridH2TableFilterCollocation> states = new HashMap<>();
+
+        for (int i = 0; i < filters.length; i++) {
+            if (getDistributedMultiplier0(filters[i].getMasks(), filters, i, states) != MULTIPLIER_COLLOCATED)
+                return false;
+        }
+
+        return true;
     }
 
     /**
@@ -180,15 +208,35 @@ public abstract class GridH2IndexBase extends BaseIndex {
 
         Map<TableFilter,GridH2TableFilterCollocation> states = qctx.tableFilterStateCache();
 
-        assert states != null;
-
         // Need to do this clean up because subquery states can be outdated here.
         clearPreviousSubQueryStates(filters, filter, states);
 
+        return getDistributedMultiplier0(masks, filters, filter, states);
+    }
+
+    /**
+     * @param masks Masks.
+     * @param filters All joined table filters.
+     * @param filter Current filter.
+     * @param states States map.
+     * @return Multiplier.
+     */
+    private static int getDistributedMultiplier0(int[] masks, TableFilter[] filters, int filter,
+        Map<TableFilter,GridH2TableFilterCollocation> states) {
+        assert states != null;
+
         final TableFilter f = filters[filter];
 
+        if (!(f.getTable() instanceof GridH2Table)) {
+            GridH2TableFilterCollocation state = getStateForNonTable(f, states);
+
+            return state.isCollocated() ? MULTIPLIER_COLLOCATED : MULTIPLIER_BROADCAST;
+        }
+
+        GridH2Table tbl = (GridH2Table)f.getTable();
+
         // Only partitioned tables will do distributed joins.
-        if (!getTable().isPartitioned()) {
+        if (!tbl.isPartitioned()) {
             states.put(f, REPLICATED);
 
             return MULTIPLIER_COLLOCATED;
@@ -204,7 +252,7 @@ public abstract class GridH2IndexBase extends BaseIndex {
         }
 
         // If we don't have affinity equality conditions then most probably we will have to broadcast.
-        if (!hasEqualityCondition(masks, affinityColumn())) {
+        if (!hasEqualityCondition(masks, affinityColumn(tbl))) {
             states.put(f, PARTITIONED_NOT_COLLOCATED);
 
             return MULTIPLIER_BROADCAST;
@@ -229,10 +277,10 @@ public abstract class GridH2IndexBase extends BaseIndex {
      * @return {@code true} If the given filter is joined with previous partitioned table filter which is
      *      also collocated. Thus the whole join chain will be collocated.
      */
-    private boolean joinedWithCollocated(TableFilter f, Map<TableFilter,GridH2TableFilterCollocation> states) {
+    private static boolean joinedWithCollocated(TableFilter f, Map<TableFilter,GridH2TableFilterCollocation> states) {
         ArrayList<IndexCondition> idxConditions = f.getIndexConditions();
 
-        int affColId = affinityColumn();
+        int affColId = affinityColumn((GridH2Table)f.getTable());
 
         for (int i = 0; i < idxConditions.size(); i++) {
             IndexCondition c = idxConditions.get(i);
@@ -447,10 +495,11 @@ public abstract class GridH2IndexBase extends BaseIndex {
     }
 
     /**
+     * @param tbl Table.
      * @return Affinity column.
      */
-    protected int affinityColumn() {
-        return getTable().getAffinityKeyColumn().column.getColumnId();
+    protected static int affinityColumn(GridH2Table tbl) {
+        return tbl.getAffinityKeyColumn().column.getColumnId();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e16d0d8/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
index 82e6f73..b5243a7 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
@@ -127,15 +127,6 @@ public class GridH2QueryContext {
     }
 
     /**
-     * @return {@code true} If it was proved that the query is fully collocated.
-     */
-    public boolean tryProveCollocated() {
-        assert type() == PREPARE : type();
-
-        return GridH2IndexBase.tryProveCollocated(tableFilterStateCache);
-    }
-
-    /**
      * @param distributedJoins Distributed joins can be run in this query.
      * @return {@code this}.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e16d0d8/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
index 28d27ff..774e110 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
@@ -907,7 +907,7 @@ public class GridH2Table extends TableBase {
         @Override public double getCost(Session ses, int[] masks, TableFilter[] filters, int filter,
             SortOrder sortOrder) {
             long rows = getRowCountApproximation();
-            int mul = delegate.getDistributedMultiplier(masks, filters, filter);
+            int mul = delegate.getDistributedMultiplier(ses, masks, filters, filter);
 
             return  mul * (rows + Constants.COST_ROW_OFFSET);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e16d0d8/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index 41de44c..ed00e48 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -682,7 +682,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
         if (qctx == null || !qctx.distributedJoins())
             return null;
 
-        int affColId = affinityColumn();
+        int affColId = affinityColumn(getTable());
         int[] masks = filter.getMasks();
         boolean unicast = masks != null && masks[affColId] == IndexCondition.EQUALITY;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e16d0d8/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
index 35da48d..6cb4bb9 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
@@ -232,12 +232,12 @@ public class GridSqlQueryParser {
      * @param stmt Prepared statement.
      * @return Parsed select.
      */
-    public static GridSqlQuery parse(JdbcPreparedStatement stmt) {
+    public static Prepared prepared(JdbcPreparedStatement stmt) {
         Command cmd = COMMAND.get(stmt);
 
         assert cmd instanceof CommandContainer;
 
-        return new GridSqlQueryParser().parse(PREPARED.get(cmd));
+        return PREPARED.get(cmd);
     }
 
     /**
@@ -361,6 +361,20 @@ public class GridSqlQueryParser {
     }
 
     /**
+     * @param qry Prepared.
+     * @return Query.
+     */
+    public static Query query(Prepared qry) {
+        if (qry instanceof Query)
+            return (Query)qry;
+
+        if (qry instanceof Explain)
+            return query(EXPLAIN_COMMAND.get((Explain)qry));
+
+        throw new CacheException("Unsupported query: " + qry);
+    }
+
+    /**
      * @param qry Select.
      */
     public GridSqlQuery parse(Prepared qry) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e16d0d8/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
index 10342b5..9225967 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
@@ -30,17 +30,21 @@ import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgnitePredicate;
+import org.h2.command.Prepared;
 import org.h2.jdbc.JdbcPreparedStatement;
 import org.h2.table.Column;
 import org.h2.table.IndexColumn;
 import org.h2.util.IntArray;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase.isCollocated;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.AVG;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.CAST;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.COUNT;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.SUM;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlPlaceholder.EMPTY;
+import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser.prepared;
+import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser.query;
 
 /**
  * Splits a single SQL query into two step map-reduce query.
@@ -139,13 +143,13 @@ public class GridSqlQuerySplitter {
     /**
      * @param stmt Prepared statement.
      * @param params Parameters.
-     * @param collocated Collocated query.
+     * @param collocatedGroupBy Whether the query has collocated GROUP BY keys.
      * @return Two step query.
      */
     public static GridCacheTwoStepQuery split(
         JdbcPreparedStatement stmt,
         Object[] params,
-        boolean collocated
+        boolean collocatedGroupBy
     ) {
         if (params == null)
             params = GridCacheSqlQuery.EMPTY_PARAMS;
@@ -153,7 +157,11 @@ public class GridSqlQuerySplitter {
         Set<String> spaces = new HashSet<>();
         Set<String> tbls = new HashSet<>();
 
-        GridSqlQuery qry = collectAllTables(GridSqlQueryParser.parse(stmt), spaces, tbls);
+        final Prepared prepared = prepared(stmt);
+
+        GridSqlQuery qry = new GridSqlQueryParser().parse(prepared);
+
+        qry = collectAllTables(qry, spaces, tbls);
 
         // Build resulting two step query.
         GridCacheTwoStepQuery res = new GridCacheTwoStepQuery(spaces, tbls);
@@ -163,10 +171,15 @@ public class GridSqlQuerySplitter {
         // nullifying or updating things, have to make sure that we will not need them in the original form later.
         final GridSqlSelect mapQry = wrapUnion(qry);
 
-        GridCacheSqlQuery rdc = split(res, 0, mapQry, params, collocated);
+        GridCacheSqlQuery rdc = split(res, 0, mapQry, params, collocatedGroupBy);
 
         res.reduceQuery(rdc);
 
+        // We do not have to look at each map query separately here, because if
+        // the whole initial query is collocated, then all the map sub-queries
+        // will be collocated as well.
+        res.fullCollocation(isCollocated(query(prepared)));
+
         return res;
     }
 
@@ -258,11 +271,11 @@ public class GridSqlQuerySplitter {
      * @param splitIdx Split index.
      * @param mapQry Map query to be split.
      * @param params Query parameters.
-     * @param collocated Whether the query is forced to be collocated.
+     * @param collocatedGroupBy Whether the query has collocated GROUP BY keys.
      * @return Reduce query for the given map query.
      */
     public static GridCacheSqlQuery split(GridCacheTwoStepQuery res, int splitIdx, final GridSqlSelect mapQry,
-        Object[] params, boolean collocated) {
+        Object[] params, boolean collocatedGroupBy) {
         final boolean explain = mapQry.explain();
 
         mapQry.explain(false);
@@ -280,7 +293,7 @@ public class GridSqlQuerySplitter {
         boolean aggregateFound = false;
 
         for (int i = 0, len = mapExps.size(); i < len; i++) // Remember len because mapExps list can grow.
-            aggregateFound |= splitSelectExpression(mapExps, rdcExps, colNames, i, collocated);
+            aggregateFound |= splitSelectExpression(mapExps, rdcExps, colNames, i, collocatedGroupBy);
 
         // -- SELECT
         mapQry.clearColumns();
@@ -301,11 +314,11 @@ public class GridSqlQuerySplitter {
         findAffinityColumnConditions(mapQry.where());
 
         // -- GROUP BY
-        if (mapQry.groupColumns() != null && !collocated)
+        if (mapQry.groupColumns() != null && !collocatedGroupBy)
             rdcQry.groupColumns(mapQry.groupColumns());
 
         // -- HAVING
-        if (mapQry.havingColumn() >= 0 && !collocated) {
+        if (mapQry.havingColumn() >= 0 && !collocatedGroupBy) {
             // TODO IGNITE-1140 - Find aggregate functions in HAVING clause or rewrite query to put all aggregates to SELECT clause.
             rdcQry.whereAnd(column(columnName(mapQry.havingColumn())));
 


[35/50] [abbrv] ignite git commit: Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-split2

Posted by sb...@apache.org.
Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-split2

# Conflicts:
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
#	modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
#	modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
#	modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
#	modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
#	modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java


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

Branch: refs/heads/ignite-1232
Commit: 5303fb822ac4860ca7fdec52cb8f99f7e75513b7
Parents: d661395 18c413c
Author: S.Vladykin <sv...@gridgain.com>
Authored: Sat Dec 26 08:13:57 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Sat Dec 26 08:13:57 2015 +0300

----------------------------------------------------------------------
 README.txt                                      |   4 +-
 RELEASE_NOTES.txt                               |   1 +
 assembly/dependencies-fabric-lgpl.xml           |   7 +
 assembly/dependencies-fabric.xml                |   7 +
 examples/pom.xml                                |   2 +-
 examples/schema-import/pom.xml                  |   2 +-
 .../hibernate/CacheHibernateStoreExample.java   |   8 +
 .../datagrid/store/hibernate/Person.hbm.xml     |   6 +-
 .../datagrid/store/hibernate/hibernate.cfg.xml  |   7 +-
 ...ComputeClientBinaryTaskExecutionExample.java |   4 +-
 .../CacheClientBinaryPutGetExample.java         |   6 +-
 .../datagrid/CacheClientBinaryQueryExample.java |  10 +-
 .../datagrid/CacheEntryProcessorExample.java    | 157 ++++++
 .../examples/datagrid/CacheQueryExample.java    |  15 +-
 .../store/jdbc/CacheJdbcStoreExample.java       |   1 -
 .../store/spring/CacheSpringPersonStore.java    | 118 +++++
 .../store/spring/CacheSpringStoreExample.java   | 155 ++++++
 .../datagrid/store/spring/package-info.java     |  22 +
 .../apache/ignite/examples/model/Address.java   |  72 +++
 .../apache/ignite/examples/model/Employee.java  |  93 ++++
 .../ignite/examples/model/EmployeeKey.java      |  93 ++++
 .../ignite/examples/model/Organization.java     |  85 +++-
 .../ignite/examples/model/OrganizationType.java |  32 ++
 .../apache/ignite/examples/model/Person.java    |   2 +-
 .../ignite/examples/model/binary/Address.java   |  72 ---
 .../ignite/examples/model/binary/Employee.java  |  93 ----
 .../examples/model/binary/EmployeeKey.java      |  93 ----
 .../examples/model/binary/Organization.java     |  93 ----
 .../examples/model/binary/OrganizationType.java |  32 --
 .../ignite/examples/model/package-info.java     |  23 +
 .../streaming/StreamVisitorExample.java         |  40 +-
 .../datagrid/CacheEntryProcessorExample.java    | 147 ++++++
 .../ScalarCacheEntryProcessorExample.scala      | 125 +++++
 .../ignite/examples/CacheExamplesSelfTest.java  |   8 +
 .../java8/examples/CacheExamplesSelfTest.java   |   8 +
 .../tests/examples/ScalarExamplesSelfTest.scala |   5 +
 modules/aop/pom.xml                             |   2 +-
 modules/apache-license-gen/pom.xml              |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/camel/pom.xml                           |   2 +-
 modules/clients/pom.xml                         |   2 +-
 .../ClientAbstractConnectivitySelfTest.java     |   2 +-
 .../JettyRestProcessorAbstractSelfTest.java     |  25 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../binary/BinaryReflectiveSerializer.java      |  33 ++
 .../ignite/binary/BinaryTypeConfiguration.java  |  20 +-
 .../org/apache/ignite/cache/CacheManager.java   |   2 +-
 .../ignite/cache/affinity/AffinityKey.java      |   2 +-
 .../configuration/CacheConfiguration.java       |  44 +-
 .../apache/ignite/internal/IgniteKernal.java    |  35 +-
 .../ignite/internal/IgniteVersionUtils.java     |   5 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |  51 +-
 .../internal/binary/BinaryClassDescriptor.java  | 181 ++++---
 .../ignite/internal/binary/BinaryContext.java   | 161 ++++--
 .../internal/binary/BinaryEnumObjectImpl.java   |   4 +-
 .../internal/binary/BinaryFieldAccessor.java    |   3 -
 .../internal/binary/BinaryObjectImpl.java       |  47 +-
 .../ignite/internal/binary/BinaryUtils.java     |  60 ++-
 .../ignite/internal/binary/BinaryWriteMode.java |   4 +-
 .../internal/binary/GridBinaryMarshaller.java   |  92 +++-
 .../connection/GridClientNioTcpConnection.java  |  17 +-
 .../internal/cluster/ClusterGroupAdapter.java   |  10 +-
 .../managers/communication/GridIoManager.java   |  39 +-
 .../discovery/GridDiscoveryManager.java         |   4 +-
 .../processors/cache/CacheEntryImpl.java        |   6 +-
 .../processors/cache/CacheEntryImplEx.java      |  17 +-
 .../processors/cache/CacheObjectAdapter.java    |   3 +
 .../processors/cache/GridCacheAdapter.java      |  47 +-
 .../processors/cache/GridCacheProcessor.java    |  11 +-
 .../processors/cache/GridCacheProxyImpl.java    |  29 ++
 .../cache/GridCacheSharedContext.java           |  18 +-
 .../processors/cache/GridCacheUtils.java        |  18 +-
 .../processors/cache/IgniteCacheProxy.java      |  43 +-
 .../processors/cache/IgniteInternalCache.java   |  26 +
 .../cache/binary/BinaryMetadataKey.java         |   2 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |  12 +-
 .../dht/CacheDistributedGetFutureAdapter.java   |   2 +-
 .../distributed/dht/GridDhtLockFuture.java      |   2 +-
 .../dht/GridPartitionedGetFuture.java           |  15 +-
 .../dht/GridPartitionedSingleGetFuture.java     |   8 +
 .../dht/atomic/GridDhtAtomicCache.java          |   8 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  | 160 +++---
 .../colocated/GridDhtColocatedLockFuture.java   |  11 +-
 .../GridDhtPartitionsExchangeFuture.java        |  20 +-
 .../dht/preloader/GridDhtPreloader.java         |  31 +-
 .../distributed/near/GridNearGetFuture.java     |  17 +-
 .../distributed/near/GridNearLockFuture.java    |  39 +-
 ...arOptimisticSerializableTxPrepareFuture.java |   5 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   5 +-
 ...ridNearOptimisticTxPrepareFutureAdapter.java |  22 +-
 .../query/GridCacheDistributedQueryManager.java |   6 +-
 .../cache/query/GridCacheQueryManager.java      |   8 +-
 .../transactions/IgniteTxLocalAdapter.java      |   2 +
 .../cache/transactions/IgniteTxManager.java     |  65 ++-
 .../datastreamer/DataStreamProcessor.java       |  12 +-
 .../datastreamer/DataStreamerUpdateJob.java     |   2 +-
 .../PlatformDotNetConfigurationClosure.java     |   4 +-
 .../processors/query/GridQueryProcessor.java    | 219 ++++++--
 .../client/message/GridClientCacheBean.java     | 139 +++++
 .../rest/client/message/GridClientNodeBean.java |  70 +--
 .../top/GridTopologyCommandHandler.java         |  38 +-
 .../processors/task/GridTaskWorker.java         |   8 +-
 .../ignite/internal/util/IgniteUtils.java       |   6 +-
 .../ignite/internal/util/lang/GridFunc.java     |   1 +
 .../internal/util/lang/GridNodePredicate.java   |  13 +-
 .../util/nio/GridNioRecoveryDescriptor.java     |   7 -
 .../ignite/internal/util/nio/GridNioServer.java |  18 +-
 .../util/nio/GridSelectorNioSessionImpl.java    |   7 +
 .../cache/VisorCacheQueryConfiguration.java     |  11 +
 .../cache/VisorCacheStoreConfiguration.java     |  13 +-
 .../internal/visor/query/VisorQueryJob.java     |   6 +
 .../internal/visor/util/VisorTaskUtils.java     | 122 ++++-
 .../ignite/lang/IgniteProductVersion.java       |   2 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  12 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  48 +-
 .../TcpDiscoveryMulticastIpFinder.java          |  12 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../test/config/websession/example-cache.xml    |   9 +-
 .../ignite/IgniteCacheAffinitySelfTest.java     |   7 -
 .../fair/FairAffinityDynamicCacheSelfTest.java  |  17 +-
 .../ignite/internal/ClusterGroupSelfTest.java   |  32 +-
 .../IgniteClientReconnectCacheTest.java         |  33 +-
 ...niteClientReconnectFailoverAbstractTest.java |   3 +-
 .../ignite/internal/TaskNodeRestartTest.java    | 230 +++++++++
 .../binary/BinaryFieldsAbstractSelfTest.java    |   4 +-
 .../BinaryFooterOffsetsAbstractSelfTest.java    |   3 +-
 .../binary/BinaryMarshallerSelfTest.java        | 219 ++++++--
 ...GridBinaryMarshallerCtxDisabledSelfTest.java |   3 +-
 .../binary/GridBinaryWildcardsSelfTest.java     |   3 +-
 .../cache/CacheStopAndDestroySelfTest.java      |   2 +
 .../cache/CrossCacheTxRandomOperationsTest.java |   2 -
 .../cache/GridCacheAbstractFullApiSelfTest.java |  22 +-
 .../cache/GridCacheAbstractSelfTest.java        |   3 +-
 .../GridCacheOnCopyFlagAbstractSelfTest.java    |  33 +-
 ...IgniteCacheBinaryEntryProcessorSelfTest.java |   4 +-
 .../IgniteCacheBinaryObjectsScanSelfTest.java   | 137 +++++
 .../cache/IgniteCacheNearLockValueSelfTest.java |  11 +-
 .../cache/IgniteCachePutAllRestartTest.java     |   2 +-
 ...iteCacheScanPredicateDeploymentSelfTest.java | 114 +++++
 .../cache/IgniteCacheSerializationSelfTest.java | 112 ++++
 .../IgniteCacheStoreValueAbstractTest.java      |   2 +-
 .../IgniteStartCacheInTransactionSelfTest.java  |  39 +-
 .../GridBinaryCacheEntryMemorySizeSelfTest.java |   3 +-
 .../CacheGetInsideLockChangingTopologyTest.java | 477 +++++++++++++++++
 ...yMetadataUpdateChangingTopologySelfTest.java | 104 ++--
 ...niteBinaryMetadataUpdateNodeRestartTest.java | 411 +++++++++++++++
 .../distributed/IgniteCacheManyClientsTest.java |   2 +
 .../IgniteCacheSizeFailoverTest.java            |   4 +-
 .../random/RandomEvictionPolicySelfTest.java    |   4 +-
 ...ContinuousQueryFailoverAbstractSelfTest.java | 128 ++---
 ...ridCacheContinuousQueryAbstractSelfTest.java |   3 +
 .../service/ClosureServiceClientsNodesTest.java |  22 +-
 .../GridServiceProcessorStopSelfTest.java       |  35 +-
 .../product/GridProductVersionSelfTest.java     |  22 +-
 .../internal/util/nio/GridNioSelfTest.java      | 127 +++--
 .../IgniteMessagingWithClientTest.java          |   2 -
 .../GridSessionCheckpointAbstractSelfTest.java  |   3 +-
 .../session/GridSessionCheckpointSelfTest.java  |   3 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |  21 +
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |  17 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |  16 +-
 .../TcpDiscoveryMulticastIpFinderSelfTest.java  |  21 +-
 .../testframework/GridSpiTestContext.java       |  18 +-
 .../ignite/testframework/junits/IgniteMock.java |   3 +-
 .../junits/IgniteTestResources.java             |   4 +-
 .../IgniteCacheFailoverTestSuite3.java          |   2 +
 .../IgniteCacheRestartTestSuite2.java           |   3 +
 .../testsuites/IgniteCacheTestSuite3.java       |   2 +
 .../testsuites/IgniteComputeGridTestSuite.java  |   2 +
 modules/extdata/p2p/pom.xml                     |   2 +-
 .../p2p/CacheDeploymentAlwaysTruePredicate.java |  29 ++
 .../ignite/tests/p2p/cache/PersonWrapper.java   | 121 +++++
 .../extdata/uri/modules/uri-dependency/pom.xml  |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/flume/pom.xml                           |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 .../processors/query/h2/IgniteH2Indexing.java   | 105 +++-
 .../processors/query/h2/sql/GridSqlQuery.java   |   3 +
 .../query/h2/sql/GridSqlQuerySplitter.java      |  72 ++-
 .../cache/BinarySerializationQuerySelfTest.java | 416 +++++++++++++++
 ...onQueryWithReflectiveSerializerSelfTest.java |  28 +
 .../IgniteBinaryObjectFieldsQuerySelfTest.java  |  69 ++-
 ...eBinaryWrappedObjectFieldsQuerySelfTest.java |  28 +
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |  68 ++-
 .../cache/IgniteCacheAbstractQuerySelfTest.java | 299 +++++++----
 ...teCacheFullTextQueryNodeJoiningSelfTest.java | 145 ++++++
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |   4 +-
 .../IgniteCacheReplicatedQuerySelfTest.java     |  10 +-
 .../local/IgniteCacheLocalQuerySelfTest.java    |   2 +-
 .../query/IgniteSqlSchemaIndexingTest.java      | 240 +++++++++
 .../query/IgniteSqlSplitterSelfTest.java        |  51 ++
 .../query/h2/sql/GridQueryParsingTest.java      |   9 +-
 .../IgniteBinaryCacheQueryTestSuite.java        | 128 ++++-
 .../IgniteCacheQuerySelfTestSuite.java          |  11 +
 modules/jcl/pom.xml                             |   2 +-
 modules/jms11/pom.xml                           |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/kafka/pom.xml                           |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/log4j2/pom.xml                          |   2 +-
 modules/mesos/pom.xml                           |   2 +-
 modules/mqtt/pom.xml                            |   2 +-
 .../stream/mqtt/IgniteMqttStreamerTest.java     |  33 +-
 modules/osgi-karaf/pom.xml                      |   2 +-
 modules/osgi-paxlogging/pom.xml                 |   2 +-
 modules/osgi/pom.xml                            |   2 +-
 modules/platforms/cpp/common/configure.ac       |   2 +-
 modules/platforms/cpp/core-test/configure.ac    |   2 +-
 modules/platforms/cpp/core/configure.ac         |   2 +-
 modules/platforms/cpp/core/impl/doxygen.h       |  53 --
 .../cpp/core/include/ignite/binary/binary.h     |   5 +
 .../core/include/ignite/binary/binary_consts.h  |   5 +
 .../include/ignite/binary/binary_containers.h   |   5 +
 .../include/ignite/binary/binary_raw_reader.h   |   5 +
 .../include/ignite/binary/binary_raw_writer.h   |   5 +
 .../core/include/ignite/binary/binary_reader.h  |   5 +
 .../core/include/ignite/binary/binary_type.h    |  17 +
 .../core/include/ignite/binary/binary_writer.h  |   5 +
 .../cpp/core/include/ignite/cache/cache.h       |   5 +
 .../cpp/core/include/ignite/cache/cache_entry.h |   5 +
 .../core/include/ignite/cache/cache_peek_mode.h |   5 +
 .../cpp/core/include/ignite/cache/query/query.h |   5 +
 .../include/ignite/cache/query/query_argument.h |   6 +
 .../include/ignite/cache/query/query_cursor.h   |   5 +
 .../ignite/cache/query/query_fields_cursor.h    |   5 +
 .../ignite/cache/query/query_fields_row.h       |   5 +
 .../include/ignite/cache/query/query_scan.h     |   5 +
 .../core/include/ignite/cache/query/query_sql.h |   5 +
 .../ignite/cache/query/query_sql_fields.h       |   5 +
 .../include/ignite/cache/query/query_text.h     |   5 +
 .../platforms/cpp/core/include/ignite/guid.h    |   5 +
 .../platforms/cpp/core/include/ignite/ignite.h  |   5 +
 .../core/include/ignite/ignite_configuration.h  |   5 +
 .../cpp/core/include/ignite/ignite_error.h      |   5 +
 .../cpp/core/include/ignite/ignition.h          |   5 +
 modules/platforms/cpp/core/namespaces.dox       |  53 ++
 modules/platforms/cpp/examples/configure.ac     |   2 +-
 modules/platforms/cpp/ignite/configure.ac       |   2 +-
 .../Properties/AssemblyInfo.cs                  |   5 +-
 .../Properties/AssemblyInfo.cs                  |   5 +-
 .../Properties/AssemblyInfo.cs                  |   5 +-
 .../Services/ServicesTest.cs                    |  23 +-
 .../Binary/IBinaryRawReader.cs                  |   4 +-
 .../Apache.Ignite.Core/Binary/IBinaryReader.cs  |  25 +-
 .../Apache.Ignite.Core/Binary/Package-Info.cs   |   2 +-
 .../Apache.Ignite.Core/Cache/Package-Info.cs    |   2 +-
 .../Cache/Query/Continuous/Package-Info.cs      |   2 +-
 .../Cache/Query/Package-Info.cs                 |   2 +-
 .../Cache/Store/Package-Info.cs                 |   2 +-
 .../Apache.Ignite.Core/Cluster/Package-Info.cs  |   2 +-
 .../Apache.Ignite.Core/Compute/Package-Info.cs  |   2 +-
 .../Datastream/Package-Info.cs                  |   2 +-
 .../Apache.Ignite.Core/Events/Package-Info.cs   |   2 +-
 .../Impl/Binary/BinaryReader.cs                 |  13 +-
 .../Impl/Binary/BinaryUtils.cs                  |   4 +-
 .../Lifecycle/Package-Info.cs                   |   2 +-
 .../Messaging/Package-Info.cs                   |   2 +-
 .../dotnet/Apache.Ignite.Core/Package-Info.cs   |   2 +-
 .../Properties/AssemblyInfo.cs                  |   5 +-
 .../Apache.Ignite.Core/Resource/Package-Info.cs |   2 +-
 .../Apache.Ignite.Core/Services/Package-Info.cs |   2 +-
 .../Transactions/Package-Info.cs                |   2 +-
 modules/platforms/dotnet/Apache.Ignite.dxg      |   2 +-
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |   5 +-
 .../Messaging/MessagingExample.cs               |   6 +-
 .../Properties/AssemblyInfo.cs                  |   5 +-
 .../Properties/AssemblyInfo.cs                  |   5 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar-2.10/pom.xml                     |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 .../ignite/schema/generator/CodeGenerator.java  |   7 +-
 .../schema/test/AbstractSchemaImportTest.java   |  18 +
 .../org/apache/ignite/schema/test/model/Tst.txt | 506 +++++++++++++++++++
 .../apache/ignite/schema/test/model/TstKey.txt  |  96 ++++
 .../schema/test/model/ignite-type-metadata.xml  | 180 +++++++
 .../test/parser/DbMetadataParserTest.java       |  17 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spark-2.10/pom.xml                      |   2 +-
 modules/spark/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 .../java/org/apache/ignite/IgniteSpring.java    |   4 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/twitter/pom.xml                         |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 .../GridUriDeploymentFileProcessorSelfTest.java |  19 +-
 modules/visor-console-2.10/pom.xml              |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 .../commands/cache/VisorCacheCommand.scala      |  14 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 .../cache/websession/WebSessionFilter.java      |  82 +--
 .../cache/websession/WebSessionListener.java    |  25 +-
 .../internal/websession/WebSessionSelfTest.java |   2 -
 .../config/benchmark-multicast.properties       |   6 +-
 modules/yardstick/pom.xml                       |   2 +-
 .../cache/IgnitePutTxImplicitBenchmark.java     |  52 ++
 .../cache/IgnitePutTxOffHeapBenchmark.java      |   2 +-
 .../IgnitePutTxOffHeapValuesBenchmark.java      |   2 +-
 modules/yarn/pom.xml                            |   2 +-
 modules/zookeeper/pom.xml                       |   2 +-
 pom.xml                                         |  44 +-
 310 files changed, 7989 insertions(+), 1874 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5303fb82/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/5303fb82/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 9d3cd31,4808e96..643968f
--- 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
@@@ -64,8 -42,10 +42,9 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.cache.CacheObjectContext;
  import org.apache.ignite.internal.processors.cache.GridCacheContext;
  import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
+ import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
  import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture;
  import org.apache.ignite.internal.processors.cache.query.CacheQueryType;
 -import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
  import org.apache.ignite.internal.util.GridSpinBusyLock;
  import org.apache.ignite.internal.util.future.GridCompoundFuture;
  import org.apache.ignite.internal.util.future.GridFinishedFuture;
@@@ -185,15 -190,19 +189,21 @@@ public class GridQueryProcessor extend
      }
  
      /**
 -     * @param ccfg Cache configuration.
 +     * @param cctx Cache context.
       * @throws IgniteCheckedException If failed.
       */
 -    public void initializeCache(CacheConfiguration<?, ?> ccfg) throws IgniteCheckedException {
 -        idx.registerCache(ccfg);
 +    private void initializeCache(GridCacheContext<?, ?> cctx) throws IgniteCheckedException {
 +        CacheConfiguration<?,?> ccfg = cctx.config();
 +
 +        idx.registerCache(cctx, cctx.config());
  
          try {
+             List<Class<?>> mustDeserializeClss = null;
+ 
+             boolean binaryEnabled = ctx.cacheObjects().isBinaryEnabled(ccfg);
+ 
+             CacheObjectContext coCtx = binaryEnabled ? ctx.cacheObjects().contextForCache(ccfg) : null;
+ 
              if (!F.isEmpty(ccfg.getQueryEntities())) {
                  for (QueryEntity qryEntity : ccfg.getQueryEntities()) {
                      if (F.isEmpty(qryEntity.getValueType()))
@@@ -245,15 -274,9 +275,15 @@@
  
                          if (valCls != null)
                              altTypeId = new TypeId(ccfg.getName(), valCls);
 +
 +                        // Need to setup affinity key for distributed joins.
 +                        String affField = ctx.cacheObjects().affinityField(qryEntity.getKeyType());
 +
 +                        if (affField != null)
 +                            desc.affinityKey(affField);
                      }
                      else {
-                         processClassMeta(qryEntity, desc);
+                         processClassMeta(qryEntity, desc, coCtx);
  
                          typeId = new TypeId(ccfg.getName(), valCls);
                          altTypeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(qryEntity.getValueType()));

http://git-wip-us.apache.org/repos/asf/ignite/blob/5303fb82/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/5303fb82/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/5303fb82/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 23e7ec4,be72888..90dcc88
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@@ -247,10 -242,10 +253,13 @@@ public class IgniteH2Indexing implement
      /** */
      private GridReduceQueryExecutor rdcQryExec;
  
+     /** space name -> schema name */
+     private final Map<String, String> space2schema = new ConcurrentHashMap8<>();
+ 
      /** */
 +    private GridSpinBusyLock busyLock;
 +
 +    /** */
      private final ThreadLocal<ConnectionWrapper> connCache = new ThreadLocal<ConnectionWrapper>() {
          @Nullable @Override public ConnectionWrapper get() {
              ConnectionWrapper c = super.get();
@@@ -1650,13 -1563,16 +1697,15 @@@
      }
  
      /** {@inheritDoc} */
-     @Override public void registerCache(GridCacheContext<?,?> cctx, CacheConfiguration<?,?> ccfg)
 -    @Override public void registerCache(CacheConfiguration<?,?> ccfg) throws IgniteCheckedException {
++    @Override public void registerCache(GridCacheContext<?,?> cctx, CacheConfiguration<?,?> ccfg) 
 +        throws IgniteCheckedException {
-         String schema = schema(ccfg.getName());
+         String schema = schemaNameFromCacheConf(ccfg);
  
 -        if (schemas.putIfAbsent(schema, new Schema(ccfg.getName(), schema,
 -            ccfg.getOffHeapMaxMemory() >= 0 || ccfg.getMemoryMode() == CacheMemoryMode.OFFHEAP_TIERED ?
 -            new GridUnsafeMemory(0) : null, ccfg)) != null)
 -            throw new IgniteCheckedException("Schema for cache already registered: " + U.maskName(ccfg.getName()));
 +        if (schemas.putIfAbsent(schema, new Schema(ccfg.getName(), cctx, ccfg)) != null)
 +            throw new IgniteCheckedException("Cache already registered: " + U.maskName(ccfg.getName()));
  
+         space2schema.put(emptyIfNull(ccfg.getName()), schema);
+ 
          createSchema(schema);
  
          createSqlFunctions(schema, ccfg.getSqlFunctionClasses());
@@@ -2405,24 -2194,19 +2456,26 @@@
  
          /**
           * @param spaceName Space name.
+          * @param schemaName Schema name.
 -         * @param offheap Offheap memory.
 +         * @param cctx Cache context.
           * @param ccfg Cache configuration.
           */
 -        private Schema(@Nullable String spaceName, String schemaName, GridUnsafeMemory offheap, CacheConfiguration<?,?> ccfg) {
 +        private Schema(@Nullable String spaceName, GridCacheContext<?,?> cctx, CacheConfiguration<?,?> ccfg) {
              this.spaceName = spaceName;
 +            this.cctx = cctx;
+             this.schemaName = schemaName;
 -            this.offheap = offheap;
              this.ccfg = ccfg;
  
 -            if (offheap != null)
 -                rowCache = new CacheLongKeyLIRS<>(ccfg.getSqlOnheapRowCacheSize(), 1, 128, 256);
 -            else
 +            offheap = ccfg.getOffHeapMaxMemory() >= 0 || ccfg.getMemoryMode() == CacheMemoryMode.OFFHEAP_TIERED ?
 +                new GridUnsafeMemory(0) : null;
 +
 +            if (offheap != null) {
 +                CacheLongKeyLIRS.Config lirsCfg = new CacheLongKeyLIRS.Config();
 +
 +                lirsCfg.maxMemory = ccfg.getSqlOnheapRowCacheSize();
 +
 +                rowCache = new CacheLongKeyLIRS<>(lirsCfg);
 +            } else
                  rowCache = null;
          }
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/5303fb82/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
index a50c210,3d9c10a..08f12d9
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
@@@ -157,18 -145,8 +160,18 @@@ public class GridSqlQuerySplitter 
          if (params == null)
              params = GridCacheSqlQuery.EMPTY_PARAMS;
  
-         Set<String> spaces = new HashSet<>();
 +        Set<String> tbls = new HashSet<>();
+         Set<String> schemas = new HashSet<>();
  
 +        final Prepared prepared = prepared(stmt);
 +
 +        GridSqlQuery qry = new GridSqlQueryParser().parse(prepared);
 +
-         qry = collectAllTables(qry, spaces, tbls);
++        qry = collectAllTables(qry, schemas, tbls);
 +
 +        // Build resulting two step query.
 +        GridCacheTwoStepQuery res = new GridCacheTwoStepQuery(spaces, tbls);
 +
          // Map query will be direct reference to the original query AST.
          // Thus all the modifications will be performed on the original AST, so we should be careful when
          // nullifying or updating things, have to make sure that we will not need them in the original form later.
@@@ -296,34 -172,40 +302,46 @@@
          boolean aggregateFound = false;
  
          for (int i = 0, len = mapExps.size(); i < len; i++) // Remember len because mapExps list can grow.
-             aggregateFound |= splitSelectExpression(mapExps, rdcExps, colNames, i, collocatedGroupBy);
 -            aggregateFound |= splitSelectExpression(mapExps, rdcExps, colNames, i, collocated, i == havingCol);
++            aggregateFound |= splitSelectExpression(mapExps, rdcExps, colNames, i, collocatedGroupBy, i == havingCol);
  
 -        // Fill select expressions.
 +        // -- SELECT
          mapQry.clearColumns();
  
          for (GridSqlElement exp : mapExps) // Add all map expressions as visible.
              mapQry.addColumn(exp, true);
  
-         for (GridSqlElement rdcExp : rdcExps) // Add corresponding visible reduce columns.
-             rdcQry.addColumn(rdcExp, true);
+         for (int i = 0; i < visibleCols; i++) // Add visible reduce columns.
+             rdcQry.addColumn(rdcExps.get(i), true);
+ 
+         for (int i = visibleCols; i < rdcExps.size(); i++) // Add invisible reduce columns (HAVING).
+             rdcQry.addColumn(rdcExps.get(i), false);
  
-         for (int i = rdcExps.length; i < mapExps.size(); i++)  // Add all extra map columns as invisible reduce columns.
+         for (int i = rdcExps.size(); i < mapExps.size(); i++)  // Add all extra map columns as invisible reduce columns.
              rdcQry.addColumn(column(((GridSqlAlias)mapExps.get(i)).alias()), false);
  
 +        // -- FROM
 +        findAffinityColumnConditions(mapQry.from());
 +
 +        // -- WHERE
 +        findAffinityColumnConditions(mapQry.where());
 +
          // -- GROUP BY
 -        if (mapQry.groupColumns() != null && !collocated)
 +        if (mapQry.groupColumns() != null && !collocatedGroupBy)
              rdcQry.groupColumns(mapQry.groupColumns());
  
          // -- HAVING
-         if (mapQry.havingColumn() >= 0 && !collocatedGroupBy) {
 -        if (havingCol >= 0 && !collocated) {
++        if (havingCol >= 0 && !collocatedGroupBy) {
              // TODO IGNITE-1140 - Find aggregate functions in HAVING clause or rewrite query to put all aggregates to SELECT clause.
-             rdcQry.whereAnd(column(columnName(mapQry.havingColumn())));
+             // We need to find HAVING column in reduce query.
+             for (int i = visibleCols; i < rdcQry.allColumns(); i++) {
+                 GridSqlElement c = rdcQry.column(i);
+ 
+                 if (c instanceof GridSqlAlias && HAVING_COLUMN.equals(((GridSqlAlias)c).alias())) {
+                     rdcQry.havingColumn(i);
+ 
+                     break;
+                 }
+             }
  
              mapQry.havingColumn(-1);
          }
@@@ -418,11 -315,10 +436,12 @@@
  
      /**
       * @param qry Query.
 +     * @param spaces Space names.
 +     * @param tbls Tables.
+      * @param schemas Shemas' names.
       * @return Query.
       */
 -    private static GridSqlQuery collectAllSpaces(GridSqlQuery qry, Set<String> schemas) {
 +    private static GridSqlQuery collectAllTables(GridSqlQuery qry, Set<String> spaces, Set<String> tbls) {
          if (qry instanceof GridSqlUnion) {
              GridSqlUnion union = (GridSqlUnion)qry;
  
@@@ -445,44 -341,10 +464,45 @@@
  
      /**
       * @param from From element.
 +     * @param spaces Space names.
 +     * @param tbls Tables.
+      * @param schemas Shemas' names.
       */
 -    private static void collectAllSpacesInFrom(GridSqlElement from, Set<String> schemas) {
 -        assert from != null;
 +    private static void collectAllTablesInFrom(GridSqlElement from, final Set<String> spaces, final Set<String> tbls) {
 +        findTablesInFrom(from, new IgnitePredicate<GridSqlElement>() {
 +            @Override public boolean apply(GridSqlElement el) {
 +                if (el instanceof GridSqlTable) {
 +                    GridSqlTable tbl = (GridSqlTable)el;
 +
 +                    String schema = tbl.schema();
 +
 +                    if (schema != null && spaces != null)
 +                        spaces.add(IgniteH2Indexing.space(schema));
 +
 +                    if (tbls != null)
 +                        tbls.add(tbl.dataTable().identifier());
 +                }
 +                else if (el instanceof GridSqlSubquery)
 +                    collectAllTables(((GridSqlSubquery)el).select(), spaces, tbls);
 +
 +                return false;
 +            }
 +        });
 +    }
 +
 +    /**
 +     * Processes all the tables and subqueries using the given closure.
 +     *
 +     * @param from FROM element.
 +     * @param c Closure each found table and subquery will be passed to. If returns {@code true} the we need to stop.
 +     * @return {@code true} If we have found.
 +     */
 +    private static boolean findTablesInFrom(GridSqlElement from, IgnitePredicate<GridSqlElement> c) {
 +        if (from == null)
 +            return false;
 +
 +        if (from instanceof GridSqlTable || from instanceof GridSqlSubquery)
 +            return c.apply(from);
  
          if (from instanceof GridSqlJoin) {
              // Left and right.
@@@ -506,14 -368,11 +526,15 @@@
      /**
       * Searches spaces in subqueries in SELECT and WHERE clauses.
       * @param el Element.
 +     * @param spaces Space names.
 +     * @param tbls Tables.
+      * @param schemas Schemas' names.
       */
 -    private static void collectAllSpacesInSubqueries(GridSqlElement el, Set<String> schemas) {
 -        if (el instanceof GridSqlAlias)
 -            el = el.child();
 +    private static void collectAllTablesInSubqueries(GridSqlElement el, Set<String> spaces, Set<String> tbls) {
 +        if (el == null)
 +            return;
 +
 +        el = GridSqlAlias.unwrap(el);
  
          if (el instanceof GridSqlOperation || el instanceof GridSqlFunction) {
              for (GridSqlElement child : el)

http://git-wip-us.apache.org/repos/asf/ignite/blob/5303fb82/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
index cdb926e,1507543..4ca6877
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
@@@ -64,7 -65,10 +65,9 @@@ import org.apache.ignite.configuration.
  import org.apache.ignite.events.CacheQueryExecutedEvent;
  import org.apache.ignite.events.CacheQueryReadEvent;
  import org.apache.ignite.events.Event;
+ import org.apache.ignite.internal.IgniteKernal;
+ import org.apache.ignite.internal.binary.BinaryMarshaller;
  import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQuerySelfTest;
 -import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager;
  import org.apache.ignite.internal.processors.cache.query.QueryCursorEx;
  import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
  import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@@ -759,8 -811,24 +810,8 @@@ public abstract class IgniteCacheAbstra
      /**
       * @throws Exception If failed.
       */
 -    public void testEmptyObject() throws Exception {
 -        IgniteCache<EmptyObject, EmptyObject> cache = ignite().cache(null);
 -
 -        cache.put(new EmptyObject(1), new EmptyObject(2));
 -
 -        for (int i = 0; i < gridCount(); i++) {
 -            GridCacheQueryManager<Object, Object> qryMgr =
 -                ((IgniteKernal)grid(i)).internalCache().context().queries();
 -
 -            assert !hasIndexTable(EmptyObject.class, qryMgr);
 -        }
 -    }
 -
 -    /**
 -     * @throws Exception If failed.
 -     */
      public void testPrimitiveType() throws Exception {
-         IgniteCache<Integer, Integer> cache = ignite.cache(null);
+         IgniteCache<Integer, Integer> cache = ignite().cache(null);
  
          cache.put(1, 1);
          cache.put(2, 2);
@@@ -1262,20 -1378,29 +1361,29 @@@
  
                      return true;
                  }
-             }, EVT_CACHE_QUERY_EXECUTED);
+             };
+ 
+             grid(i).events().localListen(pred, EVT_CACHE_QUERY_EXECUTED);
+             qryExecLsnrs[i] = pred;
          }
  
-         IgniteCache<UUID, Person> cache = ignite.cache(null);
+         try {
 -            IgniteCache<UUID, Person> cache = ignite().cache(null);
++            IgniteCache<UUID,Person> cache = ignite().cache(null);
  
-         for (int i = 1; i <= 20; i++)
-             cache.put(UUID.randomUUID(), new Person("Person " + i, i));
+             for (int i = 1; i <= 20; i++)
+                 cache.put(UUID.randomUUID(), new Person("Person " + i, i));
  
-         QueryCursor<List<?>> q = cache.query(new SqlFieldsQuery("select _key, name from Person where salary > ?").
-             setArgs(10));
+             QueryCursor<List<?>> q = cache.query(new SqlFieldsQuery("select _key, name from Person where salary > ?").
+                 setArgs(10));
  
-         q.getAll();
+             q.getAll();
  
-         assert execLatch.await(1000, MILLISECONDS);
+             assert execLatch.await(1000, MILLISECONDS);
+         }
+         finally {
+             for (int i = 0; i < gridCount(); i++)
+                 grid(i).events().stopLocalListen(qryExecLsnrs[i]);
+         }
      }
  
      /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/5303fb82/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
index 942a95c,d0e2780..b996861
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
@@@ -209,88 -207,55 +212,136 @@@ public class IgniteSqlSplitterSelfTest 
      }
  
      /**
 +     * @throws Exception If failed.
 +     */
 +    public void testDistributedJoins() throws Exception {
 +        CacheConfiguration ccfg = cacheConfig("persOrg", true,
 +            Integer.class, Person.class, Integer.class, Organization.class);
 +
 +        IgniteCache<Integer, Object> c = ignite(0).getOrCreateCache(ccfg);
 +
 +        try {
 +            awaitPartitionMapExchange();
 +
 +            doTestDistributedJoins(c, 30, 100, 1000, false);
 +            doTestDistributedJoins(c, 30, 100, 1000, true);
 +
 +            doTestDistributedJoins(c, 3, 10, 3, false);
 +            doTestDistributedJoins(c, 3, 10, 3, true);
 +
 +            doTestDistributedJoins(c, 300, 2000, 5, false);
 +            doTestDistributedJoins(c, 300, 2000, 5, true);
 +        }
 +        finally {
 +            c.destroy();
 +        }
 +    }
 +
 +    /**
+      * Test HAVING clause.
+      */
+     public void testHaving() {
+         IgniteCache<Integer, Integer> c = ignite(0).getOrCreateCache(cacheConfig("ints", true,
+             Integer.class, Integer.class));
+ 
+         try {
+             Random rnd = new GridRandom();
+ 
+             Map<Integer, AtomicLong> cntMap = new HashMap<>();
+ 
+             for (int i = 0; i < 1000; i++) {
+                 int v = (int)(50 * rnd.nextGaussian());
+ 
+                 c.put(i, v);
+ 
+                 AtomicLong cnt = cntMap.get(v);
+ 
+                 if (cnt == null)
+                     cntMap.put(v, cnt = new AtomicLong());
+ 
+                 cnt.incrementAndGet();
+             }
+ 
+             assertTrue(cntMap.size() > 10);
+ 
+             String sqlQry = "select _val, count(*) cnt from Integer group by _val having cnt > ?";
+ 
+             X.println("Plan: " + c.query(new SqlFieldsQuery("explain " + sqlQry).setArgs(0)).getAll());
+ 
+             for (int i = -1; i <= 1001; i += 10) {
+                 List<List<?>> res = c.query(new SqlFieldsQuery(sqlQry).setArgs(i)).getAll();
+ 
+                 for (List<?> row : res) {
+                     int v = (Integer)row.get(0);
+                     long cnt = (Long)row.get(1);
+ 
+                     assertTrue(cnt + " > " + i, cnt > i);
+                     assertEquals(cntMap.get(v).longValue(), cnt);
+                 }
+             }
+         }
+         finally {
+             c.destroy();
+         }
+     }
+ 
+     /**
       * @param c Cache.
 +     * @param orgs Number of organizations.
 +     * @param persons Number of persons.
 +     * @param pageSize Page size.
 +     * @param enforceJoinOrder Enforce join order.
 +     */
 +    private void doTestDistributedJoins(IgniteCache<Integer, Object> c, int orgs, int persons, int pageSize,
 +        boolean enforceJoinOrder) {
 +        assertEquals(0, c.size(CachePeekMode.ALL));
 +
 +        int key = 0;
 +
 +        for (int i = 0; i < orgs; i++) {
 +            Organization o = new Organization();
 +
 +            o.name = "Org" + i;
 +
 +            c.put(key++, o);
 +        }
 +
 +        Random rnd = new GridRandom();
 +
 +        for (int i = 0; i < persons; i++) {
 +            Person p = new Person();
 +
 +            p.name = "Person" + i;
 +            p.orgId = rnd.nextInt(orgs);
 +
 +            c.put(key++, p);
 +        }
 +
 +        String select = "select count(*) from Organization o, Person p where p.orgId = o._key";
 +
 +        String plan = (String)c.query(new SqlFieldsQuery("explain " + select)
 +            .setDistributedJoins(true).setEnforceJoinOrder(enforceJoinOrder).setPageSize(pageSize))
 +            .getAll().get(0).get(0);
 +
 +        X.println("Plan : " + plan);
 +
 +        if (enforceJoinOrder)
 +            assertTrue(plan, plan.contains("batched:broadcast"));
 +        else
 +            assertTrue(plan, plan.contains("batched:unicast"));
 +
 +        assertEquals(Long.valueOf(persons), c.query(new SqlFieldsQuery(select).setDistributedJoins(true)
 +            .setEnforceJoinOrder(enforceJoinOrder).setPageSize(pageSize)).getAll().get(0).get(0));
 +
 +        c.clear();
 +
 +        assertEquals(0, c.size(CachePeekMode.ALL));
 +        assertEquals(0L, c.query(new SqlFieldsQuery(select).setDistributedJoins(true)
 +            .setEnforceJoinOrder(enforceJoinOrder).setPageSize(pageSize)).getAll().get(0).get(0));
 +    }
 +
 +    /**
 +     * @param c Cache.
       * @param qry Query.
       * @param args Arguments.
       * @return Column as list.

http://git-wip-us.apache.org/repos/asf/ignite/blob/5303fb82/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/5303fb82/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
index e3d763a,eddfcf4..de94d53
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
@@@ -62,27 -119,76 +119,75 @@@ public class IgniteBinaryCacheQueryTest
  
          TestSuite suite = new TestSuite("Grid Cache Query Test Suite using BinaryMarshaller");
  
-         // Parsing
+         // Serialization.
+         suite.addTestSuite(BinarySerializationQuerySelfTest.class);
+         suite.addTestSuite(BinarySerializationQueryWithReflectiveSerializerSelfTest.class);
+ 
+         // Parsing.
          suite.addTestSuite(GridQueryParsingTest.class);
  
+         // Config.
+         suite.addTestSuite(IgniteCacheDuplicateEntityConfigurationSelfTest.class);
+ 
          // Queries tests.
+         suite.addTestSuite(IgniteSqlSplitterSelfTest.class);
          suite.addTestSuite(GridCacheQueryIndexDisabledSelfTest.class);
+         suite.addTestSuite(IgniteCacheQueryLoadSelfTest.class);
+         suite.addTestSuite(IgniteCacheLocalQuerySelfTest.class);
+         suite.addTestSuite(IgniteCacheLocalAtomicQuerySelfTest.class);
+         suite.addTestSuite(IgniteCacheReplicatedQuerySelfTest.class);
+         suite.addTestSuite(IgniteCacheReplicatedQueryP2PDisabledSelfTest.class);
+         suite.addTestSuite(IgniteCachePartitionedQuerySelfTest.class);
+         suite.addTestSuite(IgniteCachePartitionedSnapshotEnabledQuerySelfTest.class);
+         suite.addTestSuite(IgniteCacheAtomicQuerySelfTest.class);
+         suite.addTestSuite(IgniteCacheAtomicNearEnabledQuerySelfTest.class);
+         suite.addTestSuite(IgniteCachePartitionedQueryP2PDisabledSelfTest.class);
          suite.addTestSuite(IgniteCachePartitionedQueryMultiThreadedSelfTest.class);
+         suite.addTestSuite(IgniteCacheQueryIndexSelfTest.class);
+         suite.addTestSuite(IgniteCacheCollocatedQuerySelfTest.class);
          suite.addTestSuite(IgniteCacheLargeResultSelfTest.class);
+         suite.addTestSuite(GridCacheQueryInternalKeysSelfTest.class);
          suite.addTestSuite(IgniteCacheQueryMultiThreadedSelfTest.class);
+         suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.class);
          suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
          suite.addTestSuite(IgniteCacheQueryOffheapMultiThreadedSelfTest.class);
- 
+         suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class);
+         suite.addTestSuite(IgniteCacheOffheapEvictQueryTest.class);
+         suite.addTestSuite(IgniteCacheSqlQueryMultiThreadedSelfTest.class);
          suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);
+         suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class);
+         suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest2.class);
+         suite.addTestSuite(IgniteCacheClientQueryReplicatedNodeRestartSelfTest.class);
 -        suite.addTestSuite(GridCacheReduceQueryMultithreadedSelfTest.class);
+         suite.addTestSuite(GridCacheCrossCacheQuerySelfTest.class);
+         suite.addTestSuite(GridCacheQuerySerializationSelfTest.class);
+         suite.addTestSuite(IgniteCacheBinaryObjectsScanSelfTest.class);
+         suite.addTestSuite(IgniteBinaryObjectFieldsQuerySelfTest.class);
+         suite.addTestSuite(IgniteBinaryWrappedObjectFieldsQuerySelfTest.class);
+ 
+         // Scan queries.
+         suite.addTestSuite(CacheScanPartitionQueryFallbackSelfTest.class);
  
          // Fields queries.
+         suite.addTestSuite(SqlFieldsQuerySelfTest.class);
+         suite.addTestSuite(IgniteCacheLocalFieldsQuerySelfTest.class);
+         suite.addTestSuite(IgniteCacheReplicatedFieldsQuerySelfTest.class);
+         suite.addTestSuite(IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest.class);
+         suite.addTestSuite(IgniteCachePartitionedFieldsQuerySelfTest.class);
+         suite.addTestSuite(IgniteCacheAtomicFieldsQuerySelfTest.class);
+         suite.addTestSuite(IgniteCacheAtomicNearEnabledFieldsQuerySelfTest.class);
+         suite.addTestSuite(IgniteCachePartitionedFieldsQueryP2PEnabledSelfTest.class);
          suite.addTestSuite(IgniteCacheFieldsQueryNoDataSelfTest.class);
-         suite.addTestSuite(IgniteBinaryObjectFieldsQuerySelfTest.class);
  
          // Continuous queries.
+         suite.addTestSuite(GridCacheContinuousQueryLocalSelfTest.class);
          suite.addTestSuite(GridCacheContinuousQueryLocalAtomicSelfTest.class);
+         suite.addTestSuite(GridCacheContinuousQueryReplicatedSelfTest.class);
          suite.addTestSuite(GridCacheContinuousQueryReplicatedAtomicSelfTest.class);
+         suite.addTestSuite(GridCacheContinuousQueryReplicatedP2PDisabledSelfTest.class);
+         suite.addTestSuite(GridCacheContinuousQueryPartitionedSelfTest.class);
          suite.addTestSuite(GridCacheContinuousQueryPartitionedOnlySelfTest.class);
+         suite.addTestSuite(GridCacheContinuousQueryPartitionedP2PDisabledSelfTest.class);
+         suite.addTestSuite(GridCacheContinuousQueryTxSelfTest.class);
          suite.addTestSuite(GridCacheContinuousQueryAtomicSelfTest.class);
          suite.addTestSuite(GridCacheContinuousQueryAtomicNearEnabledSelfTest.class);
          suite.addTestSuite(GridCacheContinuousQueryAtomicP2PDisabledSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5303fb82/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 5e191cc,4b1eafa..5bef70b
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@@ -112,7 -120,7 +120,8 @@@ public class IgniteCacheQuerySelfTestSu
  
          // Queries tests.
          suite.addTestSuite(IgniteSqlSplitterSelfTest.class);
+         suite.addTestSuite(IgniteSqlSchemaIndexingTest.class);
 +        suite.addTestSuite(IgniteCachePartitionedQuerySelfTest.class);
          suite.addTestSuite(GridCacheQueryIndexDisabledSelfTest.class);
          suite.addTestSuite(IgniteCacheQueryLoadSelfTest.class);
          suite.addTestSuite(IgniteCacheLocalQuerySelfTest.class);
@@@ -139,8 -148,11 +148,10 @@@
          suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class);
          suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest2.class);
          suite.addTestSuite(IgniteCacheClientQueryReplicatedNodeRestartSelfTest.class);
 -        suite.addTestSuite(GridCacheReduceQueryMultithreadedSelfTest.class);
          suite.addTestSuite(GridCacheCrossCacheQuerySelfTest.class);
          suite.addTestSuite(GridCacheQuerySerializationSelfTest.class);
+         suite.addTestSuite(IgniteBinaryObjectFieldsQuerySelfTest.class);
+         suite.addTestSuite(IgniteBinaryWrappedObjectFieldsQuerySelfTest.class);
  
          // Scan queries.
          suite.addTestSuite(CacheScanPartitionQueryFallbackSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5303fb82/modules/spring/pom.xml
----------------------------------------------------------------------


[32/50] [abbrv] ignite git commit: ignite-split2 - fixes

Posted by sb...@apache.org.
ignite-split2 - fixes


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

Branch: refs/heads/ignite-1232
Commit: b1a38a9ffb36c1e63bbbb9b4bffdb549fd5f064a
Parents: 3cf7587
Author: S.Vladykin <sv...@gridgain.com>
Authored: Thu Dec 17 04:02:44 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Thu Dec 17 04:02:44 2015 +0300

----------------------------------------------------------------------
 .../dht/GridDhtPartitionsReservation.java       |  3 +-
 .../processors/query/h2/IgniteH2Indexing.java   | 25 +++++++--
 .../query/h2/opt/GridH2QueryContext.java        | 48 ++++++++++++++--
 .../query/h2/opt/GridH2TreeIndex.java           | 27 ++++++---
 .../query/h2/twostep/GridMapQueryExecutor.java  | 58 +++++++++++---------
 .../h2/twostep/GridReduceQueryExecutor.java     |  5 ++
 ...QueryNodeRestartDistributedJoinSelfTest.java | 34 ++++++------
 7 files changed, 137 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b1a38a9f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
index d12247e..2f51c5a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
@@ -204,7 +204,8 @@ public class GridDhtPartitionsReservation implements GridReservable {
 
             if (reservations.compareAndSet(r, r - 1)) {
                 // If it was the last reservation and topology version changed -> attempt to evict partitions.
-                if (r == 1 && !topVer.equals(cctx.topology().topologyVersion()))
+                if (r == 1 && !cctx.kernalContext().isStopping() &&
+                    !topVer.equals(cctx.topology().topologyVersion()))
                     tryEvict(parts.get());
 
                 return;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b1a38a9f/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 7b696ff..b16b48e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -208,9 +208,12 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * Command in H2 prepared statement.
      */
     static {
-        try {
-            System.setProperty("h2.objectCache", "false");
+        // Initialize system properties for H2.
+        System.setProperty("h2.objectCache", "false");
+        System.setProperty("h2.serializeJavaObject", "false");
+        System.setProperty("h2.objectCacheMaxPerElementSize", "0"); // Avoid ValueJavaObject caching.
 
+        try {
             COMMAND_FIELD = JdbcPreparedStatement.class.getDeclaredField("command");
 
             COMMAND_FIELD.setAccessible(true);
@@ -246,6 +249,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     private GridReduceQueryExecutor rdcQryExec;
 
     /** */
+    private GridSpinBusyLock busyLock;
+
+    /** */
     private final ThreadLocal<ConnectionWrapper> connCache = new ThreadLocal<ConnectionWrapper>() {
         @Nullable @Override public ConnectionWrapper get() {
             ConnectionWrapper c = super.get();
@@ -1438,6 +1444,13 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
+     * @return Busy lock.
+     */
+    public GridSpinBusyLock busyLock() {
+        return busyLock;
+    }
+
+    /**
      * @return Map query executor.
      */
     public GridMapQueryExecutor mapQueryExecutor() {
@@ -1457,8 +1470,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if (log.isDebugEnabled())
             log.debug("Starting cache query index...");
 
-        System.setProperty("h2.serializeJavaObject", "false");
-        System.setProperty("h2.objectCacheMaxPerElementSize", "0"); // Avoid ValueJavaObject caching.
+        this.busyLock = busyLock;
 
         if (SysProperties.serializeJavaObject) {
             U.warn(log, "Serialization of Java objects in H2 was enabled.");
@@ -1562,7 +1574,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             catch (IgniteCheckedException e) {
                 ok = false;
 
-                U.warn(log, e.getMessage());
+                U.warn(log, "Failed to send message [node=" + node + ", msg=" + msg +
+                    ", errMsg=" + e.getMessage() + "]");
             }
         }
 
@@ -1649,6 +1662,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             U.error(log, "Failed to shutdown database.", e);
         }
 
+        GridH2QueryContext.clearLocalNodeStop(nodeId);
+
         if (log.isDebugEnabled())
             log.debug("Cache query index stopped.");
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b1a38a9f/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
index 2c74f6d..d39d7ab 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.query.h2.opt;
 
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentMap;
@@ -31,7 +32,6 @@ import org.apache.ignite.spi.indexing.IndexingQueryFilter;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.LOCAL;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.MAP;
 
 /**
@@ -51,6 +51,9 @@ public class GridH2QueryContext {
     @GridToStringInclude
     private Map<Long, Object> snapshots;
 
+    /** */
+    private List<GridReservable> reservations;
+
     /** Range streams for indexes. */
     private Map<Integer, Object> streams;
 
@@ -144,6 +147,16 @@ public class GridH2QueryContext {
     }
 
     /**
+     * @param reservations Reserved partitions or group reservations.
+     * @return {@code this}.
+     */
+    public GridH2QueryContext reservations(List<GridReservable> reservations) {
+        this.reservations = reservations;
+
+        return this;
+    }
+
+    /**
      * @param topVer Topology version.
      * @return {@code this}.
      */
@@ -343,7 +356,7 @@ public class GridH2QueryContext {
 
         qctx.remove();
 
-        if (!onlyThreadLoc && x.key.type != LOCAL)
+        if (!onlyThreadLoc && x.key.type == MAP)
             doClear(x.key);
     }
 
@@ -361,22 +374,47 @@ public class GridH2QueryContext {
      * @param key Context key.
      */
     private static void doClear(Key key) {
+        assert key.type == MAP : key.type;
+
         GridH2QueryContext x = qctxs.remove(key);
 
-        if (x != null && !F.isEmpty(x.snapshots)) {
+        if (x == null)
+            return;
+
+        assert x.key.equals(key);
+
+        if (!F.isEmpty(x.snapshots)) {
             for (Object snapshot : x.snapshots.values()) {
                 if (snapshot instanceof GridReservable)
                     ((GridReservable)snapshot).release();
             }
         }
+
+        List<GridReservable> r = x.reservations;
+
+        if (!F.isEmpty(r)) {
+            for (int i = 0; i < r.size(); i++)
+                r.get(i).release();
+        }
     }
 
     /**
+     * @param locNodeId Local node ID.
      * @param nodeId Dead node ID.
      */
-    public static void clearAfterDeadNode(UUID nodeId) {
+    public static void clearAfterDeadNode(UUID locNodeId, UUID nodeId) {
+        for (Key key : qctxs.keySet()) {
+            if (key.locNodeId.equals(locNodeId) && key.nodeId.equals(nodeId))
+                doClear(key);
+        }
+    }
+
+    /**
+     * @param locNodeId Local node ID.
+     */
+    public static void clearLocalNodeStop(UUID locNodeId) {
         for (Key key : qctxs.keySet()) {
-            if (key.nodeId.equals(nodeId))
+            if (key.locNodeId.equals(locNodeId))
                 doClear(key);
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b1a38a9f/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index bffe4d0..5641222 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -51,6 +51,7 @@ import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2RowRange
 import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2ValueMessage;
 import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2ValueMessageFactory;
 import org.apache.ignite.internal.util.GridEmptyIterator;
+import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.offheap.unsafe.GridOffHeapSnapTreeMap;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeGuard;
 import org.apache.ignite.internal.util.snaptree.SnapTreeMap;
@@ -208,7 +209,17 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
 
             msgLsnr = new GridMessageListener() {
                 @Override public void onMessage(UUID nodeId, Object msg) {
-                    onMessage0(nodeId, msg);
+                    GridSpinBusyLock l = desc.indexing().busyLock();
+
+                    if (!l.enterBusy())
+                        return;
+
+                    try {
+                        onMessage0(nodeId, msg);
+                    }
+                    finally {
+                        l.leaveBusy();
+                    }
                 }
             };
 
@@ -1397,7 +1408,10 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
         private GridH2IndexRangeResponse awaitForResponse() {
             assert remainingRanges > 0;
 
-            for (int attempt = 0; attempt < 50; attempt++) {
+            for (int attempt = 0;; attempt++) {
+                if (kernalContext().isStopping())
+                    throw new GridH2RetryException("Stopping node.");
+
                 GridH2IndexRangeResponse res;
 
                 try {
@@ -1434,7 +1448,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
                                 throw new GridH2RetryException("Failure on remote node.");
 
                             try {
-                                U.sleep(10 * attempt);
+                                U.sleep(20 * attempt);
                             }
                             catch (IgniteInterruptedCheckedException e) {
                                 throw new IgniteInterruptedException(e.getMessage());
@@ -1442,22 +1456,19 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
 
                             send(singletonList(node), req);
 
-                            break;
+                            continue;
 
                         case GridH2IndexRangeResponse.STATUS_ERROR:
                             throw new CacheException(res.error());
 
                         default:
-                            assert false;
+                            throw new IllegalStateException();
                     }
                 }
 
                 if (!kernalContext().discovery().alive(node))
                     throw new GridH2RetryException("Node left: " + node);
             }
-
-            // Attempts exceeded.
-            throw new CacheException("Failed to get index range from remote node, request timeout.");
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b1a38a9f/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index 878814a..d8b1180 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -143,11 +143,13 @@ public class GridMapQueryExecutor {
 
         log = ctx.log(GridMapQueryExecutor.class);
 
+        final UUID locNodeId = ctx.localNodeId();
+
         ctx.event().addLocalEventListener(new GridLocalEventListener() {
             @Override public void onEvent(final Event evt) {
                 UUID nodeId = ((DiscoveryEvent)evt).eventNode().id();
 
-                GridH2QueryContext.clearAfterDeadNode(nodeId);
+                GridH2QueryContext.clearAfterDeadNode(locNodeId, nodeId);
 
                 ConcurrentMap<Long,QueryResults> nodeRess = qryRess.remove(nodeId);
 
@@ -220,14 +222,14 @@ public class GridMapQueryExecutor {
 
         long qryReqId = msg.queryRequestId();
 
+        GridH2QueryContext.clear(ctx.localNodeId(), node.id(), qryReqId, MAP);
+
         QueryResults results = nodeRess.remove(qryReqId);
 
         if (results == null)
             return;
 
         results.cancel();
-
-        GridH2QueryContext.clear(ctx.localNodeId(), node.id(), qryReqId, MAP);
     }
 
     /**
@@ -490,7 +492,8 @@ public class GridMapQueryExecutor {
                 .partitionsMap(partsMap)
                 .distributedJoins(distributedJoins)
                 .pageSize(pageSize)
-                .topologyVersion(topVer);
+                .topologyVersion(topVer)
+                .reservations(reserved);
 
             List<GridH2Table> snapshotedTbls = null;
 
@@ -510,6 +513,9 @@ public class GridMapQueryExecutor {
 
             GridH2QueryContext.set(qctx);
 
+            // qctx is set, we have to release reservations inside of it.
+            reserved = null;
+
             h2.enforceJoinOrder(true);
 
             try {
@@ -570,14 +576,8 @@ public class GridMapQueryExecutor {
                 qr.cancel();
             }
 
-            if (X.hasCause(e, GridH2RetryException.class)) {
-                try {
-                    sendRetry(node, reqId);
-                }
-                catch (IgniteCheckedException ex) {
-                    U.warn(log, "Failed to send retry message to node: " + node);
-                }
-            }
+            if (X.hasCause(e, GridH2RetryException.class))
+                sendRetry(node, reqId);
             else {
                 U.error(log, "Failed to execute local query.", e);
 
@@ -588,9 +588,11 @@ public class GridMapQueryExecutor {
             }
         }
         finally {
-            // Release reserved partitions.
-            for (GridReservable r : reserved)
-                r.release();
+            if (reserved != null) {
+                // Release reserved partitions.
+                for (int i = 0; i < reserved.size(); i++)
+                    reserved.get(i).release();
+            }
         }
     }
 
@@ -682,22 +684,26 @@ public class GridMapQueryExecutor {
     /**
      * @param node Node.
      * @param reqId Request ID.
-     * @throws IgniteCheckedException If failed.
      */
-    private void sendRetry(ClusterNode node, long reqId) throws IgniteCheckedException {
-        boolean loc = node.isLocal();
+    private void sendRetry(ClusterNode node, long reqId) {
+        try {
+            boolean loc = node.isLocal();
 
-        GridQueryNextPageResponse msg = new GridQueryNextPageResponse(reqId,
+            GridQueryNextPageResponse msg = new GridQueryNextPageResponse(reqId,
             /*qry*/0, /*page*/0, /*allRows*/0, /*cols*/1,
-            loc ? null : Collections.<Message>emptyList(),
-            loc ? Collections.<Value[]>emptyList() : null);
+                loc ? null : Collections.<Message>emptyList(),
+                loc ? Collections.<Value[]>emptyList() : null);
 
-        msg.retry(h2.readyTopologyVersion());
+            msg.retry(h2.readyTopologyVersion());
 
-        if (loc)
-            h2.reduceQueryExecutor().onMessage(ctx.localNodeId(), msg);
-        else
-            ctx.io().send(node, GridTopic.TOPIC_QUERY, msg, QUERY_POOL);
+            if (loc)
+                h2.reduceQueryExecutor().onMessage(ctx.localNodeId(), msg);
+            else
+                ctx.io().send(node, GridTopic.TOPIC_QUERY, msg, QUERY_POOL);
+        }
+        catch (Exception e) {
+            U.warn(log, "Failed to send retry message: " + e.getMessage());
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b1a38a9f/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 00d2f27..6004629 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -607,6 +607,9 @@ public class GridReduceQueryExecutor {
                 final boolean oldStyle = oldNodesInTopology();
                 final boolean distributedJoins = qry.distributedJoins();
 
+                if (oldStyle && distributedJoins)
+                    throw new CacheException("Failed to enable distributed joins. Topology contains older data nodes.");
+
                 if (send(nodes,
                     oldStyle ?
                         new GridQueryRequest(qryReqId, r.pageSize, space, mapQrys, topVer, extraSpaces, null) :
@@ -704,6 +707,8 @@ public class GridReduceQueryExecutor {
                 }
 
                 if (retry) {
+                    send(nodes, new GridQueryCancelRequest(qryReqId), null);
+
                     if (Thread.currentThread().isInterrupted())
                         throw new IgniteInterruptedCheckedException("Query was interrupted.");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b1a38a9f/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartDistributedJoinSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartDistributedJoinSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartDistributedJoinSelfTest.java
index 15f390d..de00a2b 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartDistributedJoinSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartDistributedJoinSelfTest.java
@@ -36,6 +36,7 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.util.GridRandom;
 import org.apache.ignite.internal.util.typedef.CAX;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.X;
 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;
@@ -51,16 +52,16 @@ import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
  */
 public class IgniteCacheQueryNodeRestartDistributedJoinSelfTest extends GridCommonAbstractTest {
     /** */
-    private static final String QRY_0 = "select co.id, count(*) cnt\n" +
+    private static final String QRY_0 = "select co._key, count(*) cnt\n" +
         "from \"pe\".Person pe, \"pr\".Product pr, \"co\".Company co, \"pu\".Purchase pu\n" +
-        "where pe.id = pu.personId and pu.productId = pr.id and pr.companyId = co.id \n" +
-        "group by co.id order by cnt desc, co.id";
+        "where pe._key = pu.personId and pu.productId = pr._key and pr.companyId = co._key \n" +
+        "group by co._key order by cnt desc, co._key";
 
     /** */
-    private static final String QRY_1 = "select pr.id, co.id\n" +
+    private static final String QRY_1 = "select pr._key, co._key\n" +
         "from \"pr\".Product pr, \"co\".Company co\n" +
-        "where pr.companyId = co.id\n" +
-        "order by co.id, pr.id ";
+        "where pr.companyId = co._key\n" +
+        "order by co._key, pr._key ";
 
     /** */
     private static final int GRID_CNT = 6;
@@ -182,8 +183,8 @@ public class IgniteCacheQueryNodeRestartDistributedJoinSelfTest extends GridComm
         int duration = 90 * 1000;
         int qryThreadNum = 4;
         int restartThreadsNum = 2; // 4 + 2 = 6 nodes
-        final int nodeLifeTime = 2 * 1000;
-        final int logFreq = 10;
+        final int nodeLifeTime = 4000;
+        final int logFreq = 1;
 
         startGridsMultiThreaded(GRID_CNT);
 
@@ -191,6 +192,9 @@ public class IgniteCacheQueryNodeRestartDistributedJoinSelfTest extends GridComm
 
         fillCaches();
 
+        X.println("Plan: " + grid(0).cache("pu").query(new SqlFieldsQuery("explain " + QRY_0)
+            .setDistributedJoins(true)).getAll());
+
         final List<List<?>> pRes = grid(0).cache("pu").query(new SqlFieldsQuery(QRY_0)
             .setDistributedJoins(true)).getAll();
 
@@ -206,7 +210,6 @@ public class IgniteCacheQueryNodeRestartDistributedJoinSelfTest extends GridComm
         assertFalse(rRes.isEmpty());
 
         final AtomicInteger qryCnt = new AtomicInteger();
-
         final AtomicBoolean qrysDone = new AtomicBoolean();
 
         IgniteInternalFuture<?> fut1 = multithreadedAsync(new CAX() {
@@ -221,15 +224,14 @@ public class IgniteCacheQueryNodeRestartDistributedJoinSelfTest extends GridComm
                     }
                     while (!locks.compareAndSet(g, 0, 1));
 
-                    if (rnd.nextBoolean()) { // Partitioned query.
+                    if (rnd.nextBoolean()) {
                         IgniteCache<?,?> cache = grid(g).cache("pu");
 
                         SqlFieldsQuery qry = new SqlFieldsQuery(QRY_0).setDistributedJoins(true);
 
                         boolean smallPageSize = rnd.nextBoolean();
 
-                        if (smallPageSize)
-                            qry.setPageSize(3);
+                        qry.setPageSize(smallPageSize ? 30 : 1000);
 
                         try {
                             assertEquals(pRes, cache.query(qry).getAll());
@@ -321,16 +323,12 @@ public class IgniteCacheQueryNodeRestartDistributedJoinSelfTest extends GridComm
         info("Stopping..");
 
         restartsDone.set(true);
-
-        fut2.get();
-
-        info("Restarts stopped.");
-
         qrysDone.set(true);
 
+        fut2.get();
         fut1.get();
 
-        info("Queries stopped.");
+        info("Stopped.");
     }
 
     /** {@inheritDoc} */


[49/50] [abbrv] ignite git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-split2

Posted by sb...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-split2

# Conflicts:
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
#	modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
#	modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java


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

Branch: refs/heads/ignite-1232
Commit: d9d665628cc2107d67db19e89098621bfe3d7ed4
Parents: 50fbeee 46b6a76
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Feb 15 10:17:32 2016 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Feb 15 10:17:32 2016 +0300

----------------------------------------------------------------------
 .gitignore                                      |    4 +
 DEVNOTES.txt                                    |    4 +-
 README.md                                       |  194 +++
 doap_Ignite.rdf                                 |   67 +
 examples/pom.xml                                |    2 +-
 examples/schema-import/pom.xml                  |    2 +-
 .../hibernate/HibernateL2CacheExample.java      |   26 +-
 .../hibernate/CacheHibernateStoreExample.java   |    5 +
 .../store/auto/CacheBinaryAutoStoreExample.java |    1 +
 .../examples/datagrid/CacheAffinityExample.java |   13 +-
 .../examples/datagrid/CacheApiExample.java      |    5 +
 .../examples/datagrid/CacheAsyncApiExample.java |    5 +
 .../datagrid/CacheContinuousQueryExample.java   |    5 +
 .../datagrid/CacheDataStreamerExample.java      |    5 +
 .../datagrid/CacheEntryProcessorExample.java    |    5 +
 .../examples/datagrid/CacheEventsExample.java   |    5 +
 .../examples/datagrid/CacheQueryExample.java    |    6 +
 .../datagrid/CacheTransactionExample.java       |    6 +-
 .../starschema/CacheStarSchemaExample.java      |   10 +-
 .../store/auto/CacheAutoStoreExample.java       |    5 +
 .../store/jdbc/CacheJdbcStoreExample.java       |    5 +
 .../store/spring/CacheSpringStoreExample.java   |    5 +
 .../streaming/StreamTransformerExample.java     |    4 +
 .../streaming/StreamVisitorExample.java         |   13 +-
 .../streaming/wordcount/QueryWords.java         |   51 +-
 .../socket/WordsSocketStreamerServer.java       |   86 +-
 .../java8/datagrid/CacheAffinityExample.java    |   11 +-
 .../java8/datagrid/CacheApiExample.java         |    5 +
 .../java8/datagrid/CacheAsyncApiExample.java    |    5 +
 .../datagrid/CacheEntryProcessorExample.java    |    5 +
 .../streaming/StreamTransformerExample.java     |    9 +-
 .../java8/streaming/StreamVisitorExample.java   |    5 +
 modules/aop/pom.xml                             |    4 +-
 modules/apache-license-gen/pom.xml              |    2 +-
 modules/aws/pom.xml                             |    2 +-
 modules/benchmarks/pom.xml                      |  140 ++
 .../benchmarks/jmh/JmhAbstractBenchmark.java    |  150 ++
 .../jmh/cache/JmhCacheAbstractBenchmark.java    |  184 +++
 .../benchmarks/jmh/cache/JmhCacheBenchmark.java |  145 ++
 .../jmh/runner/JmhIdeBenchmarkRunner.java       |  236 +++
 .../internal/benchmarks/model/IntValue.java     |   91 ++
 modules/camel/pom.xml                           |    2 +-
 modules/clients/pom.xml                         |    2 +-
 .../internal/client/ClientGetAffinityTask.java  |    4 +-
 modules/cloud/pom.xml                           |    2 +-
 modules/codegen/pom.xml                         |    2 +-
 modules/core/pom.xml                            |    2 +-
 .../java/org/apache/ignite/IgniteCache.java     |   45 +-
 .../java/org/apache/ignite/IgniteCluster.java   |    7 +-
 .../apache/ignite/IgniteSystemProperties.java   |   12 +-
 .../org/apache/ignite/IgniteTransactions.java   |    4 +-
 .../ignite/binary/BinaryBasicIdMapper.java      |  167 ++
 .../ignite/binary/BinaryBasicNameMapper.java    |  141 ++
 .../apache/ignite/binary/BinaryIdMapper.java    |   19 +-
 .../apache/ignite/binary/BinaryNameMapper.java  |   47 +
 .../ignite/binary/BinaryObjectBuilder.java      |    8 +-
 .../ignite/binary/BinaryTypeConfiguration.java  |   23 +-
 .../org/apache/ignite/cache/CacheEntry.java     |    9 +-
 .../apache/ignite/cache/CacheInterceptor.java   |    5 +
 .../apache/ignite/cache/CacheMemoryMode.java    |    2 +
 .../apache/ignite/cache/affinity/Affinity.java  |   24 +-
 .../configuration/BinaryConfiguration.java      |   28 +
 .../configuration/CacheConfiguration.java       |    2 +-
 .../configuration/FileSystemConfiguration.java  |   54 +-
 .../configuration/IgniteConfiguration.java      |    8 +-
 .../org/apache/ignite/igfs/IgfsUserContext.java |   16 +-
 .../igfs/secondary/IgfsSecondaryFileSystem.java |   14 -
 .../apache/ignite/internal/GridComponent.java   |    5 +-
 .../internal/GridEventConsumeHandler.java       |    3 +-
 .../internal/GridMessageListenHandler.java      |   22 +-
 .../ignite/internal/GridUpdateNotifier.java     |  454 ------
 .../apache/ignite/internal/IgniteKernal.java    |  235 ++-
 .../ignite/internal/IgniteNodeAttributes.java   |    3 +
 .../ignite/internal/IgniteVersionUtils.java     |    4 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |    7 +-
 .../ignite/internal/MarshallerContextImpl.java  |   86 +-
 .../internal/binary/BinaryClassDescriptor.java  |   86 +-
 .../ignite/internal/binary/BinaryContext.java   |  347 ++--
 .../internal/binary/BinaryEnumObjectImpl.java   |   16 +-
 .../internal/binary/BinaryFieldAccessor.java    |   45 +-
 .../internal/binary/BinaryInternalIdMapper.java |  161 --
 .../internal/binary/BinaryInternalMapper.java   |  131 ++
 .../internal/binary/BinaryMarshaller.java       |    5 +-
 .../binary/BinaryMetadataCollector.java         |   22 +-
 .../binary/BinaryObjectOffheapImpl.java         |   34 +-
 .../internal/binary/BinaryPrimitives.java       |  117 +-
 .../internal/binary/BinaryReaderExImpl.java     |   22 +-
 .../ignite/internal/binary/BinaryUtils.java     |  130 +-
 .../ignite/internal/binary/BinaryWriteMode.java |    3 +
 .../internal/binary/BinaryWriterExImpl.java     |   64 +-
 .../internal/binary/GridBinaryMarshaller.java   |   31 +-
 .../binary/builder/BinaryBuilderSerializer.java |   32 +-
 .../binary/builder/BinaryObjectBuilderImpl.java |   14 +-
 .../binary/builder/BinaryValueWithType.java     |    3 +-
 .../streams/BinaryAbstractInputStream.java      |   53 +-
 .../streams/BinaryAbstractOutputStream.java     |   90 +-
 .../binary/streams/BinaryAbstractStream.java    |   34 -
 .../binary/streams/BinaryHeapInputStream.java   |   43 +-
 .../binary/streams/BinaryHeapOutputStream.java  |   86 +-
 .../streams/BinaryMemoryAllocatorChunk.java     |    9 +-
 .../streams/BinaryOffheapInputStream.java       |   42 +-
 .../streams/BinaryOffheapOutputStream.java      |   94 +-
 .../internal/direct/DirectMessageReader.java    |   11 +
 .../internal/direct/DirectMessageWriter.java    |   18 +
 .../direct/state/DirectMessageState.java        |   10 +
 .../stream/v1/DirectByteBufferStreamImplV1.java |  116 +-
 .../stream/v2/DirectByteBufferStreamImplV2.java |  361 ++++-
 .../ignite/internal/jdbc/JdbcConnection.java    |   14 +-
 .../managers/discovery/CustomEventListener.java |    4 +-
 .../discovery/DiscoveryCustomMessage.java       |    9 -
 .../discovery/GridDiscoveryManager.java         |   61 +-
 .../affinity/GridAffinityAssignment.java        |   36 +-
 .../affinity/GridAffinityAssignmentCache.java   |    2 +-
 .../affinity/GridAffinityProcessor.java         |   60 +-
 .../processors/cache/CacheEntryImplEx.java      |   14 +-
 .../cache/CacheEntrySerializablePredicate.java  |    3 +-
 .../cache/CacheEvictableEntryImpl.java          |    4 +-
 .../cache/CacheInvokeDirectResult.java          |    4 +-
 .../processors/cache/CacheLazyEntry.java        |    3 +
 .../processors/cache/CacheMetricsSnapshot.java  |    4 +-
 .../processors/cache/CacheObjectContext.java    |   91 +-
 .../processors/cache/CacheOperationContext.java |   43 +-
 .../cache/DynamicCacheChangeBatch.java          |    5 -
 .../processors/cache/GridCacheAdapter.java      |  553 +++++--
 .../cache/GridCacheAffinityManager.java         |   77 +-
 .../cache/GridCacheClearAllRunnable.java        |    2 +-
 .../processors/cache/GridCacheContext.java      |   70 +-
 .../processors/cache/GridCacheEntryEx.java      |   11 +-
 .../cache/GridCacheEvictionManager.java         |    7 +-
 .../processors/cache/GridCacheIoManager.java    |   23 +
 .../processors/cache/GridCacheMapEntry.java     |  232 +--
 .../processors/cache/GridCacheMvccManager.java  |   41 +-
 .../cache/GridCacheOffheapSwapEntry.java        |   24 +-
 .../processors/cache/GridCachePreloader.java    |    6 +
 .../cache/GridCachePreloaderAdapter.java        |    5 +
 .../processors/cache/GridCacheProcessor.java    |  169 +-
 .../processors/cache/GridCacheProxyImpl.java    |   62 +-
 .../processors/cache/GridCacheReturn.java       |    2 +
 .../cache/GridCacheSwapEntryImpl.java           |   61 +-
 .../processors/cache/GridCacheSwapManager.java  |   46 +-
 .../cache/GridCacheUpdateAtomicResult.java      |    4 +-
 .../processors/cache/GridCacheUtils.java        |   57 +-
 .../processors/cache/IgniteCacheProxy.java      |   96 +-
 .../processors/cache/IgniteInternalCache.java   |   85 +
 .../cache/affinity/GridCacheAffinityImpl.java   |   48 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |  166 +-
 .../distributed/GridCacheTxRecoveryFuture.java  |    2 +-
 .../GridDistributedLockResponse.java            |    2 +-
 .../GridDistributedTxFinishRequest.java         |   11 +-
 .../GridDistributedTxPrepareRequest.java        |    2 +-
 .../GridDistributedTxPrepareResponse.java       |    4 +-
 .../GridDistributedTxRemoteAdapter.java         |   12 +-
 .../dht/CacheDistributedGetFutureAdapter.java   |   55 +-
 .../dht/GridClientPartitionTopology.java        |    7 +
 .../dht/GridDhtAffinityAssignmentResponse.java  |    4 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   79 +-
 .../distributed/dht/GridDhtEmbeddedFuture.java  |   13 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |  182 ++-
 .../distributed/dht/GridDhtGetSingleFuture.java |  476 ++++++
 .../distributed/dht/GridDhtLocalPartition.java  |   76 +-
 .../distributed/dht/GridDhtLockFuture.java      |   46 +-
 .../distributed/dht/GridDhtLockRequest.java     |    2 +-
 .../distributed/dht/GridDhtPartitionState.java  |    2 +-
 .../dht/GridDhtPartitionTopology.java           |    5 +
 .../dht/GridDhtPartitionTopologyImpl.java       |   52 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |   17 +-
 .../distributed/dht/GridDhtTxFinishRequest.java |   90 +-
 .../dht/GridDhtTxFinishResponse.java            |    4 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   32 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   92 +-
 .../dht/GridDhtTxPrepareRequest.java            |    2 +-
 .../dht/GridPartitionedGetFuture.java           |  257 +--
 .../dht/GridPartitionedSingleGetFuture.java     |  244 +--
 .../dht/atomic/GridDhtAtomicCache.java          |  317 +++-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   87 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |   59 +-
 .../dht/atomic/GridDhtAtomicUpdateResponse.java |    6 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   10 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |   44 +-
 .../atomic/GridNearAtomicUpdateResponse.java    |    4 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   82 +-
 .../colocated/GridDhtColocatedLockFuture.java   |    6 +-
 .../dht/preloader/GridDhtForceKeysResponse.java |    6 +-
 .../GridDhtPartitionDemandMessage.java          |    6 +-
 .../dht/preloader/GridDhtPartitionDemander.java |   15 +-
 .../GridDhtPartitionSupplyMessageV2.java        |    6 +-
 .../GridDhtPartitionsExchangeFuture.java        |   14 +-
 .../preloader/GridDhtPartitionsFullMessage.java |    2 +-
 .../GridDhtPartitionsSingleMessage.java         |    4 +-
 .../dht/preloader/GridDhtPreloader.java         |   16 +
 .../distributed/near/GridNearAtomicCache.java   |    6 +-
 .../distributed/near/GridNearCacheAdapter.java  |    6 +-
 .../distributed/near/GridNearCacheEntry.java    |    3 +-
 .../distributed/near/GridNearGetFuture.java     |  286 ++--
 .../distributed/near/GridNearGetResponse.java   |    4 +-
 .../distributed/near/GridNearLockFuture.java    |    2 +-
 ...arOptimisticSerializableTxPrepareFuture.java |  147 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   69 +-
 ...ridNearOptimisticTxPrepareFutureAdapter.java |    6 +-
 .../GridNearPessimisticTxPrepareFuture.java     |   16 +-
 .../near/GridNearSingleGetResponse.java         |    2 +-
 .../near/GridNearTransactionalCache.java        |    9 +-
 .../near/GridNearTxFinishFuture.java            |  503 ++++--
 .../near/GridNearTxFinishRequest.java           |    5 +
 .../near/GridNearTxFinishResponse.java          |    4 +-
 .../cache/distributed/near/GridNearTxLocal.java |  138 +-
 .../near/GridNearTxPrepareFutureAdapter.java    |    8 +-
 .../near/GridNearTxPrepareResponse.java         |    4 +-
 .../processors/cache/dr/GridCacheDrInfo.java    |   49 +-
 .../cache/local/GridLocalLockFuture.java        |   19 +-
 .../local/atomic/GridLocalAtomicCache.java      |  106 +-
 .../cache/query/GridCacheQueryManager.java      |   30 +-
 .../cache/query/GridCacheQueryRequest.java      |   16 +-
 .../cache/query/GridCacheQueryResponse.java     |   18 +-
 .../continuous/CacheContinuousQueryHandler.java |   30 +-
 .../CacheContinuousQueryListener.java           |    2 +-
 .../continuous/CacheContinuousQueryManager.java |  120 +-
 .../cache/transactions/IgniteInternalTx.java    |    6 +
 .../cache/transactions/IgniteTxAdapter.java     |   47 +-
 .../cache/transactions/IgniteTxEntry.java       |   55 +-
 .../cache/transactions/IgniteTxHandler.java     |   26 +-
 .../transactions/IgniteTxLocalAdapter.java      |  321 ++--
 .../cache/transactions/IgniteTxLocalEx.java     |    3 +-
 .../cache/transactions/IgniteTxManager.java     |   42 +-
 .../version/GridCachePlainVersionedEntry.java   |    2 +-
 .../cache/version/GridCacheVersionManager.java  |   23 +-
 .../IgniteCacheObjectProcessorImpl.java         |    9 +-
 .../processors/cluster/ClusterProcessor.java    |  193 ++-
 .../processors/cluster/GridUpdateNotifier.java  |  457 ++++++
 .../continuous/AbstractContinuousMessage.java   |   10 +-
 .../continuous/GridContinuousHandler.java       |    4 +-
 .../continuous/GridContinuousProcessor.java     |   88 +-
 .../datastreamer/DataStreamerRequest.java       |    1 +
 .../datastructures/DataStructuresProcessor.java |   11 +-
 .../GridCacheAtomicReferenceImpl.java           |   68 +-
 .../processors/hadoop/HadoopPayloadAware.java   |   28 +
 .../processors/igfs/IgfsAckMessage.java         |    4 +-
 .../ignite/internal/processors/igfs/IgfsEx.java |   13 -
 .../internal/processors/igfs/IgfsImpl.java      |   28 +-
 .../internal/processors/igfs/IgfsPaths.java     |   62 +-
 .../internal/processors/igfs/IgfsProcessor.java |   17 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |   11 -
 .../platform/PlatformAbstractTarget.java        |   31 +-
 .../processors/platform/PlatformIgnition.java   |    6 +-
 .../platform/PlatformNoopProcessor.java         |   30 +
 .../processors/platform/PlatformProcessor.java  |   53 +
 .../platform/PlatformProcessorImpl.java         |   57 +-
 .../processors/platform/PlatformTarget.java     |   22 +
 .../platform/cache/PlatformCache.java           |   22 +-
 .../callback/PlatformCallbackUtils.java         |    1 -
 .../platform/compute/PlatformCompute.java       |  164 +-
 .../cpp/PlatformCppConfigurationClosure.java    |   32 +
 .../datastructures/PlatformAtomicReference.java |  141 ++
 .../datastructures/PlatformAtomicSequence.java  |  122 ++
 .../dotnet/PlatformDotNetCacheStore.java        |   72 +-
 .../PlatformDotNetConfigurationClosure.java     |   80 +-
 .../platform/events/PlatformEvents.java         |   16 +-
 .../platform/memory/PlatformAbstractMemory.java |    6 +-
 .../PlatformBigEndianOutputStreamImpl.java      |   14 +-
 .../memory/PlatformInputStreamImpl.java         |   53 +-
 .../platform/memory/PlatformMemoryUtils.java    |  108 +-
 .../memory/PlatformOutputStreamImpl.java        |   58 +-
 .../platform/messaging/PlatformMessaging.java   |    7 +-
 .../platform/services/PlatformServices.java     |   14 +-
 .../utils/PlatformConfigurationUtils.java       |  621 ++++++++
 .../platform/utils/PlatformFutureUtils.java     |  119 +-
 .../platform/utils/PlatformListenable.java      |   47 +
 .../platform/utils/PlatformUtils.java           |   52 +-
 .../processors/query/GridQueryProcessor.java    |    7 +-
 .../processors/rest/GridRestProcessor.java      |    3 +
 .../handlers/cache/GridCacheCommandHandler.java |    6 +-
 .../handlers/log/GridLogCommandHandler.java     |  171 ++
 .../rest/request/GridRestLogRequest.java        |   10 +-
 .../service/GridServiceProcessor.java           |   17 +-
 .../processors/service/ServiceContextImpl.java  |    4 +-
 .../processors/task/GridTaskProcessor.java      |    2 +-
 .../ignite/internal/util/GridHandleTable.java   |   17 +-
 .../ignite/internal/util/GridJavaProcess.java   |    3 -
 .../internal/util/GridSpinReadWriteLock.java    |   10 +-
 .../ignite/internal/util/GridStringBuilder.java |   19 +
 .../apache/ignite/internal/util/GridUnsafe.java | 1483 +++++++++++++++++-
 .../ignite/internal/util/IgniteUtils.java       |  157 +-
 .../util/future/GridCompoundFuture.java         |  316 +---
 .../util/future/GridCompoundIdentityFuture.java |    6 +-
 .../internal/util/future/GridFutureAdapter.java |    2 +
 .../internal/util/io/GridUnsafeDataInput.java   |  145 +-
 .../internal/util/io/GridUnsafeDataOutput.java  |  165 +-
 .../ignite/internal/util/lang/GridFunc.java     |    3 +
 .../internal/util/nio/GridDirectParser.java     |   52 +-
 .../ignite/internal/util/nio/GridNioServer.java |  197 ++-
 .../util/nio/GridSelectorNioSessionImpl.java    |    2 +-
 .../util/nio/SelectedSelectionKeySet.java       |  132 ++
 .../util/offheap/unsafe/GridUnsafeMap.java      |   75 +-
 .../util/offheap/unsafe/GridUnsafeMemory.java   |  106 +-
 .../ignite/internal/visor/cache/VisorCache.java |   75 +-
 .../visor/cache/VisorCacheConfiguration.java    |   31 +-
 .../cache/VisorCacheQueryConfiguration.java     |   31 +-
 .../cache/VisorCacheQueryConfigurationV2.java   |   47 +
 .../cache/VisorCacheStoreConfiguration.java     |   39 +-
 .../cache/VisorCacheStoreConfigurationV2.java   |   48 +
 .../internal/visor/cache/VisorCacheV2.java      |   73 +
 .../visor/node/VisorIgfsConfiguration.java      |   27 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |   21 +-
 .../visor/node/VisorNodeDataCollectorTask.java  |    2 +-
 .../org/apache/ignite/lang/IgniteBiTuple.java   |    6 +-
 .../ignite/lang/IgniteProductVersion.java       |   18 +
 .../optimized/OptimizedClassDescriptor.java     |   59 +-
 .../optimized/OptimizedMarshaller.java          |    5 +-
 .../optimized/OptimizedMarshallerUtils.java     |   47 +-
 .../optimized/OptimizedObjectInputStream.java   |   25 +-
 .../optimized/OptimizedObjectOutputStream.java  |    4 +-
 .../OptimizedObjectStreamRegistry.java          |    4 +-
 .../PlatformDotNetCacheStoreFactoryNative.java  |   58 +
 .../extensions/communication/MessageWriter.java |    9 +-
 .../plugin/security/SecurityCredentials.java    |    4 +-
 .../communication/tcp/TcpCommunicationSpi.java  |    9 +-
 .../ignite/spi/discovery/DiscoverySpi.java      |    2 +
 .../ignite/spi/discovery/tcp/ClientImpl.java    |    6 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   21 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |    1 -
 .../tcp/ipfinder/TcpDiscoveryIpFinder.java      |   14 +-
 .../ipfinder/jdbc/TcpDiscoveryJdbcIpFinder.java |   48 +-
 .../TcpDiscoveryMulticastIpFinder.java          |   12 +-
 .../TcpDiscoveryCustomEventMessage.java         |    6 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |    4 +-
 .../ignite/stream/socket/SocketStreamer.java    |    3 +-
 .../resources/META-INF/classnames.properties    |   15 +-
 .../core/src/main/resources/ignite.properties   |    2 +-
 .../java/org/apache/ignite/GridTestIoUtils.java |  117 +-
 .../ignite/internal/GridAffinityMappedTest.java |    8 +-
 .../internal/GridAffinityNoCacheSelfTest.java   |  294 ++++
 .../internal/GridAffinityP2PSelfTest.java       |    8 +-
 .../ignite/internal/GridAffinitySelfTest.java   |   10 +-
 .../internal/GridNodeMetricsLogSelfTest.java    |   98 ++
 .../internal/GridUpdateNotifierSelfTest.java    |  137 --
 ...eClientReconnectContinuousProcessorTest.java |   32 +-
 ...UpdateNotifierPerClusterSettingSelfTest.java |  130 ++
 .../internal/TestRecordingCommunicationSpi.java |  157 ++
 .../binary/BinaryBasicIdMapperSelfTest.java     |   51 +
 .../binary/BinaryBasicNameMapperSelfTest.java   |   50 +
 .../BinaryConfigurationConsistencySelfTest.java |  231 +++
 .../internal/binary/BinaryEnumsSelfTest.java    |   12 +-
 .../binary/BinaryFieldsOffheapSelfTest.java     |   13 +-
 .../BinaryFooterOffsetsOffheapSelfTest.java     |   13 +-
 .../binary/BinaryMarshallerSelfTest.java        |  569 ++++++-
 .../BinaryObjectBuilderAdditionalSelfTest.java  |   95 +-
 ...naryObjectBuilderDefaultMappersSelfTest.java | 1165 ++++++++++++++
 .../binary/BinaryObjectBuilderSelfTest.java     | 1108 -------------
 ...ilderSimpleNameLowerCaseMappersSelfTest.java |   41 +
 .../BinarySimpleNameTestPropertySelfTest.java   |   94 ++
 .../binary/GridBinaryMetaDataSelfTest.java      |  371 -----
 .../binary/GridBinaryWildcardsSelfTest.java     |  338 +++-
 ...aultBinaryMappersBinaryMetaDataSelfTest.java |  389 +++++
 ...CaseBinaryMappersBinaryMetaDataSelfTest.java |   41 +
 .../internal/binary/TestMappedObject.java       |   25 +
 .../mutabletest/GridBinaryTestClasses.java      |    3 +-
 ...BuilderNonCompactDefaultMappersSelfTest.java |   30 +
 .../BinaryObjectBuilderNonCompactSelfTest.java  |   30 -
 ...mpactSimpleNameLowerCaseMappersSelfTest.java |   31 +
 .../AbstractBinaryStreamByteOrderSelfTest.java  |  464 ++++++
 .../BinaryHeapStreamByteOrderSelfTest.java      |   29 +
 .../BinaryOffheapStreamByteOrderSelfTest.java   |   31 +
 ...ByteBufferStreamImplV2ByteOrderSelfTest.java |  244 +++
 .../cache/CacheConfigurationLeakTest.java       |   62 +
 .../cache/CacheGetEntryAbstractTest.java        |  803 ++++++++++
 ...GetEntryOptimisticReadCommittedSeltTest.java |   36 +
 ...etEntryOptimisticRepeatableReadSeltTest.java |   36 +
 ...eGetEntryOptimisticSerializableSeltTest.java |   36 +
 ...etEntryPessimisticReadCommittedSeltTest.java |   36 +
 ...tEntryPessimisticRepeatableReadSeltTest.java |   36 +
 ...GetEntryPessimisticSerializableSeltTest.java |   36 +
 .../cache/CacheReadThroughRestartSelfTest.java  |   43 +-
 .../CacheSerializableTransactionsTest.java      |  142 +-
 ...cheAbstractFullApiMultithreadedSelfTest.java |   13 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  147 +-
 .../cache/GridCacheAffinityRoutingSelfTest.java |   10 +-
 .../cache/GridCacheConcurrentMapSelfTest.java   |   13 +-
 .../GridCacheConcurrentTxMultiNodeTest.java     |    4 +-
 .../GridCacheConditionalDeploymentSelfTest.java |    4 +-
 ...idCacheConfigurationConsistencySelfTest.java |   58 +-
 .../GridCacheDaemonNodeAbstractSelfTest.java    |   17 +-
 .../GridCacheDeploymentOffHeapSelfTest.java     |    4 +-
 ...ridCacheDeploymentOffHeapValuesSelfTest.java |   41 +
 .../cache/GridCacheDeploymentSelfTest.java      |   11 +-
 .../cache/GridCacheEntryMemorySizeSelfTest.java |    6 +-
 .../GridCacheInterceptorAbstractSelfTest.java   |  172 +-
 .../GridCacheOffHeapValuesEvictionSelfTest.java |  184 +++
 ...hePartitionedProjectionAffinitySelfTest.java |    8 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |    4 +-
 .../processors/cache/GridCacheStopSelfTest.java |    2 +-
 .../processors/cache/GridCacheTestEntryEx.java  |    5 +-
 .../cache/GridCacheUtilsSelfTest.java           |  136 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |  454 ++++--
 ...cheEntryListenerAtomicOffheapTieredTest.java |   32 +
 ...cheEntryListenerAtomicOffheapValuesTest.java |   32 +
 ...niteCacheEntryListenerExpiredEventsTest.java |  202 +++
 ...teCacheEntryListenerTxOffheapTieredTest.java |   32 +
 ...teCacheEntryListenerTxOffheapValuesTest.java |   32 +
 .../cache/IgniteCacheEntryListenerTxTest.java   |    1 +
 ...IgniteCacheGetCustomCollectionsSelfTest.java |  128 ++
 ...gniteCacheLoadRebalanceEvictionSelfTest.java |  188 +++
 .../cache/IgniteCacheNearLockValueSelfTest.java |   62 +-
 .../cache/IgniteCacheStoreCollectionTest.java   |  175 +++
 ...eDynamicCacheStartNoExchangeTimeoutTest.java |    7 +
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   30 +-
 ...niteDynamicCacheStartStopConcurrentTest.java |  119 ++
 ...eMarshallerCacheConcurrentReadWriteTest.java |  189 +++
 .../GridCacheBinaryObjectsAbstractSelfTest.java |   17 +-
 .../GridCacheBinaryStoreAbstractSelfTest.java   |   10 +
 ...naryStoreBinariesDefaultMappersSelfTest.java |   81 +
 .../GridCacheBinaryStoreBinariesSelfTest.java   |   66 -
 ...yStoreBinariesSimpleNameMappersSelfTest.java |   40 +
 ...eAbstractDataStructuresFailoverSelfTest.java |  311 +---
 .../IgniteClientDataStructuresAbstractTest.java |    3 +
 ...ridCachePartitionNotLoadedEventSelfTest.java |    7 +-
 .../IgniteCacheAtomicNodeRestartTest.java       |    2 +
 ...niteCacheClientNodeChangingTopologyTest.java |    4 +-
 .../distributed/IgniteCacheGetRestartTest.java  |  280 ++++
 .../distributed/IgniteCachePrimarySyncTest.java |  129 ++
 .../IgniteCacheReadFromBackupTest.java          |  427 +++++
 .../IgniteCacheSingleGetMessageTest.java        |   88 +-
 .../IgniteCrossCacheTxStoreSelfTest.java        |    1 +
 .../IgniteTxPreloadAbstractTest.java            |    2 +-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |    2 +-
 .../dht/GridCacheDhtMultiBackupTest.java        |    4 +-
 .../GridCacheDhtPreloadMessageCountTest.java    |   62 +-
 .../dht/GridCacheTxNodeFailureSelfTest.java     |   13 +-
 .../IgniteCacheCommitDelayTxRecoveryTest.java   |  376 +++++
 .../IgniteCachePutRetryAbstractSelfTest.java    |   36 +-
 ...gniteCachePutRetryTransactionalSelfTest.java |   21 +
 .../near/GridCacheGetStoreErrorSelfTest.java    |    9 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |    4 +-
 .../near/GridCacheNearReadersSelfTest.java      |   19 +-
 .../near/GridCacheNearTxMultiNodeSelfTest.java  |    4 +-
 ...titionedExplicitLockNodeFailureSelfTest.java |    6 +-
 .../GridCachePartitionedNodeRestartTest.java    |    4 +-
 ...ePartitionedOptimisticTxNodeRestartTest.java |    4 +-
 .../GridCacheRebalancingSyncSelfTest.java       |    8 +
 .../GridCacheReplicatedPreloadSelfTest.java     |  221 ++-
 .../RandomEvictionPolicyCacheSizeSelfTest.java  |   79 -
 .../random/RandomEvictionPolicySelfTest.java    |  363 -----
 ...ContinuousQueryFailoverAbstractSelfTest.java |   96 ++
 ...tomicPrimaryWriteOrderOffheapTieredTest.java |   33 +
 ...tinuousQueryFailoverTxOffheapTieredTest.java |   32 +
 .../CacheContinuousQueryLostPartitionTest.java  |  256 +++
 ...acheContinuousQueryRandomOperationsTest.java |  684 ++++++++
 ...ridCacheContinuousQueryAbstractSelfTest.java |   27 +-
 ...eContinuousQueryAtomicOffheapTieredTest.java |   32 +
 ...eContinuousQueryAtomicOffheapValuesTest.java |   32 +
 ...ntinuousQueryPartitionAtomicOneNodeTest.java |   37 +
 ...heContinuousQueryPartitionTxOneNodeTest.java |   37 +
 ...tinuousQueryReplicatedAtomicOneNodeTest.java |   31 +
 ...ontinuousQueryReplicatedOneNodeSelfTest.java |  120 --
 ...eContinuousQueryReplicatedTxOneNodeTest.java |  193 +++
 ...CacheContinuousQueryTxOffheapTieredTest.java |   32 +
 ...CacheContinuousQueryTxOffheapValuesTest.java |   32 +
 ...IgniteCacheContinuousQueryReconnectTest.java |  192 +++
 .../IgniteCacheWriteBehindNoUpdateSelfTest.java |  178 +++
 .../cluster/GridUpdateNotifierSelfTest.java     |  140 ++
 .../continuous/GridEventConsumeSelfTest.java    |    3 +
 .../processors/igfs/IgfsAbstractSelfTest.java   |    8 +-
 .../igfs/IgfsExUniversalFileSystemAdapter.java  |   11 +-
 .../igfs/IgfsProcessorValidationSelfTest.java   |   27 +
 .../processors/igfs/IgfsStreamsSelfTest.java    |    4 +-
 .../igfs/UniversalFileSystemAdapter.java        |    5 +-
 .../handlers/log/GridLogCommandHandlerTest.java |  190 +++
 .../GridServicePackagePrivateSelfTest.java      |   51 +
 .../IgniteServiceDynamicCachesSelfTest.java     |  183 +++
 .../processors/service/inner/MyService.java     |   30 +
 .../service/inner/MyServiceFactory.java         |   30 +
 .../processors/service/inner/MyServiceImpl.java |   45 +
 ...dUnsafeDataInputOutputByteOrderSelfTest.java |  249 +++
 .../internal/util/nio/GridNioSelfTest.java      |   11 +-
 .../ignite/lang/GridBasicPerformanceTest.java   |   11 +-
 .../apache/ignite/lang/GridTupleSelfTest.java   |   42 +-
 .../ignite/loadtests/dsi/GridDsiClient.java     |    4 +-
 ...namicProxySerializationMultiJvmSelfTest.java |  131 ++
 .../platform/PlatformComputeBroadcastTask.java  |    7 +
 .../platform/PlatformComputeEchoTask.java       |    6 +-
 .../GridSessionCheckpointAbstractSelfTest.java  |    6 +-
 .../tcp/GridCacheDhtLockBackupSelfTest.java     |    4 +-
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |    1 -
 ...CommunicationRecoveryAckClosureSelfTest.java |   21 +-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |    2 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |    9 +-
 .../startup/GridRandomCommandLineLoader.java    |    4 +-
 .../ignite/testframework/GridTestUtils.java     |   34 +-
 .../config/GridTestProperties.java              |    5 +-
 .../testframework/junits/GridAbstractTest.java  |   30 +-
 .../junits/common/GridCommonAbstractTest.java   |   36 +-
 .../multijvm/IgniteCacheProcessProxy.java       |   59 +-
 .../junits/multijvm/IgniteNodeRunner.java       |   16 +-
 .../junits/multijvm/IgniteProcessProxy.java     |   19 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    9 +-
 .../testsuites/IgniteBinaryBasicTestSuite.java  |   97 ++
 ...ctsSimpleNameMapperComputeGridTestSuite.java |   38 +
 .../IgniteBinaryObjectsTestSuite.java           |   40 +-
 ...iteBinarySimpleNameMapperBasicTestSuite.java |   38 +
 ...rySimpleNameMapperCacheFullApiTestSuite.java |   39 +
 .../IgniteCacheEvictionSelfTestSuite.java       |    4 -
 .../IgniteCacheRestartTestSuite2.java           |    3 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   10 +
 .../testsuites/IgniteCacheTestSuite2.java       |    2 +
 .../testsuites/IgniteCacheTestSuite3.java       |    2 +
 .../testsuites/IgniteCacheTestSuite4.java       |   21 +
 .../testsuites/IgniteCacheTestSuite5.java       |    4 +
 .../IgniteCacheTxRecoverySelfTestSuite.java     |    3 +
 .../testsuites/IgniteComputeGridTestSuite.java  |    4 +-
 .../testsuites/IgniteKernalSelfTestSuite.java   |    8 +-
 .../IgniteMarshallerSelfTestSuite.java          |    4 +
 .../testsuites/IgniteRestHandlerTestSuite.java  |    4 +-
 modules/docker/1.5.0.final/Dockerfile           |   40 +
 modules/docker/1.5.0.final/run.sh               |   50 +
 modules/docker/Dockerfile                       |    6 +-
 modules/extdata/p2p/pom.xml                     |    2 +-
 .../p2p/CacheDeploymentAffinityKeyMapper.java   |   35 +
 .../CacheDeploymentAlwaysTruePredicate2.java    |   30 +
 ...oymentCacheEntryEventSerializableFilter.java |   32 +
 .../p2p/CacheDeploymentCacheEntryListener.java  |   31 +
 ...CacheDeploymentCachePluginConfiguration.java |   74 +
 ...heDeploymentStoreSessionListenerFactory.java |   83 +
 .../tests/p2p/CacheDeploymentTestEnumValue.java |   47 +
 .../p2p/CacheDeploymentTestStoreFactory.java    |  113 ++
 .../extdata/uri/modules/uri-dependency/pom.xml  |    2 +-
 modules/extdata/uri/pom.xml                     |    2 +-
 modules/flume/pom.xml                           |    2 +-
 modules/gce/pom.xml                             |    2 +-
 modules/geospatial/pom.xml                      |   10 +-
 modules/hadoop/pom.xml                          |    2 +-
 .../hadoop/fs/BasicHadoopFileSystemFactory.java |  238 +++
 .../fs/CachingHadoopFileSystemFactory.java      |   86 +
 .../hadoop/fs/HadoopFileSystemFactory.java      |   52 +
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |  260 +--
 .../fs/KerberosHadoopFileSystemFactory.java     |  217 +++
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |  154 +-
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |  122 +-
 .../processors/hadoop/HadoopClassLoader.java    |  689 ++++++--
 .../hadoop/SecondaryFileSystemProvider.java     |  139 --
 .../hadoop/fs/HadoopFileSystemCacheUtils.java   |    8 +-
 .../hadoop/fs/HadoopLazyConcurrentMap.java      |    5 +-
 .../hadoop/shuffle/HadoopShuffleJob.java        |    5 +-
 .../hadoop/shuffle/HadoopShuffleMessage.java    |   10 +-
 .../shuffle/streams/HadoopDataOutStream.java    |    6 +-
 .../hadoop/v2/HadoopNativeCodeLoader.java       |   74 -
 .../processors/hadoop/v2/HadoopV2Context.java   |    2 +
 .../processors/hadoop/v2/HadoopV2MapTask.java   |   23 +-
 ...KerberosHadoopFileSystemFactorySelfTest.java |  121 ++
 .../ignite/igfs/Hadoop1DualAbstractTest.java    |   14 +-
 .../igfs/HadoopFIleSystemFactorySelfTest.java   |  326 ++++
 ...oopFileSystemUniversalFileSystemAdapter.java |   53 +-
 ...oopSecondaryFileSystemConfigurationTest.java |   27 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |   71 +-
 ...condaryFileSystemInitializationSelfTest.java |  214 +++
 .../hadoop/HadoopAbstractWordCountTest.java     |   46 +-
 .../hadoop/HadoopClassLoaderTest.java           |  101 +-
 .../hadoop/HadoopMapReduceEmbeddedSelfTest.java |    2 +-
 .../processors/hadoop/HadoopMapReduceTest.java  |   15 +-
 .../hadoop/HadoopSnappyFullMapReduceTest.java   |   28 +
 .../processors/hadoop/HadoopSnappyTest.java     |  102 ++
 .../processors/hadoop/HadoopTasksV2Test.java    |    2 +-
 .../hadoop/deps/CircularWIthHadoop.java         |   32 +
 .../hadoop/deps/CircularWithoutHadoop.java      |   27 +
 .../processors/hadoop/deps/WithCast.java        |   41 +
 .../hadoop/deps/WithClassAnnotation.java        |   28 +
 .../hadoop/deps/WithConstructorInvocation.java  |   31 +
 .../processors/hadoop/deps/WithExtends.java     |   27 +
 .../processors/hadoop/deps/WithField.java       |   29 +
 .../processors/hadoop/deps/WithImplements.java  |   36 +
 .../hadoop/deps/WithIndirectField.java          |   27 +
 .../processors/hadoop/deps/WithInitializer.java |   33 +
 .../processors/hadoop/deps/WithInnerClass.java  |   31 +
 .../hadoop/deps/WithLocalVariable.java          |   38 +
 .../hadoop/deps/WithMethodAnnotation.java       |   32 +
 .../hadoop/deps/WithMethodArgument.java         |   31 +
 .../hadoop/deps/WithMethodCheckedException.java |   31 +
 .../hadoop/deps/WithMethodInvocation.java       |   31 +
 .../hadoop/deps/WithMethodReturnType.java       |   31 +
 .../hadoop/deps/WithMethodRuntimeException.java |   31 +
 .../processors/hadoop/deps/WithOuterClass.java  |   38 +
 .../hadoop/deps/WithParameterAnnotation.java    |   31 +
 .../processors/hadoop/deps/WithStaticField.java |   29 +
 .../hadoop/deps/WithStaticInitializer.java      |   34 +
 .../processors/hadoop/deps/Without.java         |   25 +
 .../hadoop/examples/HadoopWordCount1Reduce.java |    1 +
 .../hadoop/examples/HadoopWordCount2.java       |   18 +-
 .../examples/HadoopWordCount2Reducer.java       |    1 +
 .../HadoopConcurrentHashMultimapSelftest.java   |    6 +-
 .../collections/HadoopSkipListSelfTest.java     |    5 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   29 +-
 modules/hibernate/pom.xml                       |    2 +-
 modules/indexing/pom.xml                        |   10 +-
 .../cache/CacheQueryBuildValueTest.java         |  144 ++
 .../CacheQueryOffheapEvictDataLostTest.java     |  138 ++
 .../CacheRandomOperationsMultithreadedTest.java |  507 ++++++
 .../cache/IgniteCacheAbstractQuerySelfTest.java |   10 +-
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |   20 +-
 .../IgniteBinaryCacheQueryTestSuite.java        |  176 +--
 ...narySimpleNameMapperCacheQueryTestSuite.java |   38 +
 .../IgniteCacheQuerySelfTestSuite.java          |   29 +-
 .../IgniteCacheWithIndexingTestSuite.java       |    2 +
 modules/jcl/pom.xml                             |    4 +-
 modules/jms11/pom.xml                           |    2 +-
 modules/jta/pom.xml                             |    2 +-
 modules/kafka/README.txt                        |  111 +-
 modules/kafka/pom.xml                           |   71 +-
 .../ignite/stream/kafka/KafkaStreamer.java      |    2 +-
 .../kafka/connect/IgniteSinkConnector.java      |   91 ++
 .../kafka/connect/IgniteSinkConstants.java      |   38 +
 .../stream/kafka/connect/IgniteSinkTask.java    |  165 ++
 .../kafka/IgniteKafkaStreamerSelfTestSuite.java |    9 +-
 .../stream/kafka/KafkaEmbeddedBroker.java       |  387 -----
 .../kafka/KafkaIgniteStreamerSelfTest.java      |   13 +-
 .../ignite/stream/kafka/SimplePartitioner.java  |   53 -
 .../ignite/stream/kafka/TestKafkaBroker.java    |  237 +++
 .../kafka/connect/IgniteSinkConnectorTest.java  |  250 +++
 .../kafka/src/test/resources/example-ignite.xml |   71 +
 modules/log4j/pom.xml                           |    2 +-
 modules/log4j2/pom.xml                          |    4 +-
 modules/mesos/pom.xml                           |    2 +-
 modules/mqtt/pom.xml                            |    2 +-
 modules/osgi-karaf/pom.xml                      |    6 +-
 .../osgi-karaf/src/main/resources/features.xml  |   12 +-
 modules/osgi-paxlogging/pom.xml                 |    2 +-
 modules/osgi/pom.xml                            |   10 +-
 modules/platforms/cpp/common/configure.ac       |    2 +-
 .../cpp/common/include/ignite/common/exports.h  |   26 +-
 .../cpp/common/include/ignite/common/java.h     |   57 +-
 .../cpp/common/project/vs/common.vcxproj        |    2 +-
 .../platforms/cpp/common/project/vs/module.def  |   23 +-
 modules/platforms/cpp/common/src/exports.cpp    |   88 +-
 modules/platforms/cpp/common/src/java.cpp       |  315 +++-
 modules/platforms/cpp/core-test/Makefile.am     |    1 +
 modules/platforms/cpp/core-test/configure.ac    |    2 +-
 .../cpp/core-test/project/vs/core-test.vcxproj  |    1 +
 .../project/vs/core-test.vcxproj.filters        |    3 +
 .../platforms/cpp/core-test/src/cache_test.cpp  |   12 +
 .../cpp/core-test/src/interop_memory_test.cpp   |   95 ++
 modules/platforms/cpp/core/configure.ac         |    2 +-
 .../include/ignite/impl/ignite_environment.h    |   19 +-
 .../cpp/core/src/impl/cache/cache_impl.cpp      |    2 +-
 .../cpp/core/src/impl/ignite_environment.cpp    |   30 +-
 modules/platforms/cpp/examples/configure.ac     |    2 +-
 modules/platforms/cpp/ignite/configure.ac       |    2 +-
 .../Apache.Ignite.Benchmarks.csproj             |    4 +-
 .../Properties/AssemblyInfo.cs                  |    8 +-
 .../Apache.Ignite.Core.Tests.TestDll.csproj     |    4 +-
 .../Properties/AssemblyInfo.cs                  |    8 +-
 .../Apache.Ignite.Core.Tests.csproj             |   12 +-
 .../Binary/BinaryBuilderSelfTest.cs             |    8 +-
 .../Binary/BinarySelfTest.cs                    |   41 +
 .../Binary/BinaryStructureTest.cs               |   38 +-
 .../Cache/CacheAbstractTest.cs                  |   93 +-
 .../Cache/CacheAffinityTest.cs                  |    2 +-
 .../Cache/CacheConfigurationTest.cs             |  538 +++++++
 .../Cache/CacheDynamicStartTest.cs              |    4 +-
 .../Cache/CacheTestAsyncWrapper.cs              |    7 +
 .../Query/CacheQueriesCodeConfigurationTest.cs  |  295 ++++
 .../Cache/Query/CacheQueriesTest.cs             |    2 +-
 .../Continuous/ContinuousQueryAbstractTest.cs   |    2 +-
 .../Cache/Store/CacheStoreSessionTest.cs        |    2 +-
 .../Cache/Store/CacheStoreTest.cs               |   23 +-
 .../Compute/CancellationTest.cs                 |  174 ++
 .../Compute/ComputeApiTest.cs                   |   22 +-
 .../Config/Compute/compute-grid2.xml            |   21 +
 .../Config/Compute/compute-grid3.xml            |   31 +-
 .../DataStructures/AtomicReferenceTest.cs       |  239 +++
 .../DataStructures/AtomicSequenceTest.cs        |  131 ++
 .../Dataload/DataStreamerTest.cs                |    4 +-
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |    2 +-
 .../Apache.Ignite.Core.Tests/FutureTest.cs      |    1 +
 .../IgniteConfigurationSectionTest.cs           |   69 +
 .../IgniteConfigurationSerializerTest.cs        |  554 +++++++
 .../IgniteConfigurationTest.cs                  |  367 +++++
 .../Apache.Ignite.Core.Tests/MarshallerTest.cs  |    4 +-
 .../Properties/AssemblyInfo.cs                  |    8 +-
 .../SerializationTest.cs                        |    2 +-
 .../Apache.Ignite.Core.Tests/TestRunner.cs      |    5 +-
 .../dotnet/Apache.Ignite.Core.Tests/app.config  |   54 +
 .../Apache.Ignite.Core.csproj                   |   54 +-
 .../Apache.Ignite.Core.ruleset                  |   26 +
 .../Binary/BinaryConfiguration.cs               |   30 +-
 .../Binary/BinaryTypeConfiguration.cs           |    5 +
 .../Binary/BinaryTypeNames.cs                   |   64 +-
 .../Apache.Ignite.Core/Binary/IBinaryObject.cs  |    8 +
 .../Binary/IBinarySerializer.cs                 |    6 +-
 .../Cache/CachePartialUpdateException.cs        |    2 +
 .../Apache.Ignite.Core/Cache/CacheResult.cs     |   41 +-
 .../Configuration/CacheAtomicWriteOrderMode.cs  |   43 +
 .../Cache/Configuration/CacheAtomicityMode.cs   |   54 +
 .../Cache/Configuration/CacheConfiguration.cs   |  601 +++++++
 .../Cache/Configuration/CacheMemoryMode.cs      |   60 +
 .../Cache/Configuration/CacheMode.cs            |   52 +
 .../Cache/Configuration/CacheRebalanceMode.cs   |   51 +
 .../CacheWriteSynchronizationMode.cs            |   45 +
 .../Cache/Configuration/QueryAlias.cs           |   59 +
 .../Cache/Configuration/QueryEntity.cs          |  401 +++++
 .../Cache/Configuration/QueryField.cs           |  109 ++
 .../Cache/Configuration/QueryIndex.cs           |  137 ++
 .../Cache/Configuration/QueryIndexField.cs      |   66 +
 .../Cache/Configuration/QueryIndexType.cs       |   40 +
 .../Configuration/QuerySqlFieldAttribute.cs     |   60 +
 .../Configuration/QueryTextFieldAttribute.cs    |   36 +
 .../Cache/Event/ICacheEntryEventFilter.cs       |    1 +
 .../Cache/Event/ICacheEntryEventListener.cs     |    1 +
 .../dotnet/Apache.Ignite.Core/Cache/ICache.cs   |    6 +
 .../Cache/Query/Continuous/ContinuousQuery.cs   |    2 +-
 .../Apache.Ignite.Core/Cache/Query/QueryBase.cs |    2 +-
 .../Cache/Query/SqlFieldsQuery.cs               |    2 +-
 .../Apache.Ignite.Core/Cache/Query/SqlQuery.cs  |    6 +-
 .../Apache.Ignite.Core/Cache/Query/TextQuery.cs |    7 +-
 .../Cache/Store/CacheStoreAdapter.cs            |    3 +
 .../Apache.Ignite.Core/Common/IFactory.cs       |   34 +
 .../Common/IgniteFutureCancelledException.cs    |   66 +
 .../Compute/ComputeJobAdapter.cs                |    2 +-
 .../Compute/ComputeJobResultPolicy.cs           |    2 -
 .../Compute/ComputeTaskAdapter.cs               |    2 +
 .../Compute/ComputeTaskSplitAdapter.cs          |   17 +-
 .../Apache.Ignite.Core/Compute/ICompute.cs      |  233 +++
 .../DataStructures/IAtomicReference.cs          |   64 +
 .../DataStructures/IAtomicSequence.cs           |   69 +
 .../Datastream/StreamTransformer.cs             |    2 +
 .../Datastream/StreamVisitor.cs                 |    2 +
 .../Discovery/IDiscoverySpi.cs                  |   32 +
 .../Discovery/Tcp/ITcpDiscoveryIpFinder.cs      |   34 +
 .../Multicast/TcpDiscoveryMulticastIpFinder.cs  |  135 ++
 .../Tcp/Static/TcpDiscoveryStaticIpFinder.cs    |   86 +
 .../Discovery/Tcp/TcpDiscoveryIpFinderBase.cs   |   78 +
 .../Discovery/Tcp/TcpDiscoverySpi.cs            |  144 ++
 .../Apache.Ignite.Core/Events/CacheEvent.cs     |    8 +-
 .../Events/CacheQueryExecutedEvent.cs           |    4 +-
 .../Events/CacheQueryReadEvent.cs               |    6 +-
 .../Events/CacheRebalancingEvent.cs             |    4 +-
 .../Events/CheckpointEvent.cs                   |    4 +-
 .../Apache.Ignite.Core/Events/DiscoveryEvent.cs |    4 +-
 .../Apache.Ignite.Core/Events/EventBase.cs      |   69 +-
 .../dotnet/Apache.Ignite.Core/IIgnite.cs        |   60 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  347 +++-
 .../IgniteConfigurationSection.cs               |   80 +
 .../IgniteConfigurationSection.xsd              |  281 ++++
 .../dotnet/Apache.Ignite.Core/Ignition.cs       |   69 +-
 .../Apache.Ignite.Core/Impl/Binary/Binary.cs    |   47 +-
 .../Impl/Binary/BinaryEnum.cs                   |    6 +
 .../Impl/Binary/BinaryFullTypeDescriptor.cs     |    2 +-
 .../Impl/Binary/BinaryObject.cs                 |   50 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |   75 +-
 .../Impl/Binary/BinaryReader.cs                 |    3 +-
 .../Impl/Binary/BinaryReaderExtensions.cs       |   19 +
 .../Impl/Binary/BinaryReflectiveSerializer.cs   |    2 +-
 .../Impl/Binary/BinarySystemHandlers.cs         |    9 +-
 .../Impl/Binary/BinaryUtils.cs                  |  173 +-
 .../Impl/Binary/BinaryWriter.cs                 |    3 +
 .../Impl/Binary/Io/BinaryHeapStream.cs          |    9 +
 .../Apache.Ignite.Core/Impl/Binary/JavaTypes.cs |   92 ++
 .../Impl/Binary/Marshaller.cs                   |   27 +-
 .../Impl/Binary/Metadata/BinaryType.cs          |    2 +
 .../Impl/Binary/SerializableObjectHolder.cs     |   13 +-
 .../Impl/Binary/Structure/BinaryStructure.cs    |    3 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |   51 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheOp.cs    |    3 +-
 .../Impl/Cache/Query/AbstractQueryCursor.cs     |    4 +-
 .../Continuous/ContinuousQueryFilterHolder.cs   |    2 +-
 .../Continuous/ContinuousQueryHandleImpl.cs     |    4 +-
 .../Impl/Cache/Query/FieldsQueryCursor.cs       |    2 +
 .../Impl/Cache/Query/QueryCursor.cs             |    2 +
 .../Impl/Cache/Store/CacheStore.cs              |   29 +-
 .../Impl/Common/BooleanLowerCaseConverter.cs    |   60 +
 .../Impl/Common/CancelledTask.cs                |   50 +
 .../Common/CopyOnWriteConcurrentDictionary.cs   |    1 +
 .../Impl/Common/DelegateConverter.cs            |   11 +-
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |   90 +-
 .../Impl/Common/FutureType.cs                   |   18 +-
 .../Common/IgniteConfigurationXmlSerializer.cs  |  407 +++++
 .../Impl/Common/LoadedAssembliesResolver.cs     |    1 +
 .../Impl/Common/TypeStringConverter.cs          |  115 ++
 .../Closure/ComputeAbstractClosureTask.cs       |    2 +
 .../Compute/Closure/ComputeMultiClosureTask.cs  |    2 +
 .../Closure/ComputeReducingClosureTask.cs       |    2 +
 .../Compute/Closure/ComputeSingleClosureTask.cs |    2 +
 .../Apache.Ignite.Core/Impl/Compute/Compute.cs  |  157 +-
 .../Impl/Compute/ComputeImpl.cs                 |   15 +-
 .../Impl/DataStructures/AtomicReference.cs      |   92 ++
 .../Impl/DataStructures/AtomicSequence.cs       |   90 ++
 .../Impl/Datastream/DataStreamerBatch.cs        |    1 +
 .../Impl/Datastream/DataStreamerImpl.cs         |    5 +-
 .../Impl/Events/EventTypeConverter.cs           |  133 ++
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |   56 +-
 .../Apache.Ignite.Core/Impl/Handle/Handle.cs    |    8 +-
 .../Impl/Handle/HandleRegistry.cs               |    1 +
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  104 ++
 .../Impl/IgniteConfigurationEx.cs               |   57 -
 .../Apache.Ignite.Core/Impl/IgniteManager.cs    |    5 +-
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |   34 +
 .../Apache.Ignite.Core/Impl/IgniteUtils.cs      |   13 +-
 .../Impl/InteropExceptionHolder.cs              |   11 +-
 .../Memory/PlatformBigEndianMemoryStream.cs     |    8 +
 .../Impl/Memory/PlatformMemory.cs               |    2 +
 .../Impl/Memory/PlatformMemoryManager.cs        |    2 +
 .../Impl/Memory/PlatformMemoryStream.cs         |  322 +++-
 .../Impl/Memory/PlatformRawMemory.cs            |    4 +-
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |   53 +-
 .../Impl/Resource/ResourceProcessor.cs          |    2 +-
 .../Impl/Resource/ResourceTypeDescriptor.cs     |    2 +-
 .../Impl/Services/ServiceProxy.cs               |    2 +
 .../Impl/Services/ServiceProxyInvoker.cs        |    1 +
 .../Impl/Transactions/TransactionsImpl.cs       |    4 +-
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |   67 +-
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |    5 +-
 .../Impl/Unmanaged/UnmanagedUtils.cs            |  152 +-
 .../Properties/AssemblyInfo.cs                  |    8 +-
 .../Services/ServiceInvocationException.cs      |    2 +
 modules/platforms/dotnet/Apache.Ignite.FxCop    |  354 +----
 modules/platforms/dotnet/Apache.Ignite.sln      |    4 +-
 .../Apache.Ignite.sln.TeamCity.DotSettings      |   30 +
 .../dotnet/Apache.Ignite/Apache.Ignite.csproj   |    5 +-
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |    8 +-
 .../Apache.Ignite.Examples.csproj               |    4 +-
 .../Properties/AssemblyInfo.cs                  |    8 +-
 .../Services/ServicesExample.cs                 |    2 +-
 .../Apache.Ignite.ExamplesDll.csproj            |    4 +-
 .../Properties/AssemblyInfo.cs                  |    8 +-
 modules/rest-http/pom.xml                       |    2 +-
 modules/scalar-2.10/pom.xml                     |    2 +-
 modules/scalar/pom.xml                          |    2 +-
 modules/schedule/pom.xml                        |    4 +-
 modules/schema-import/pom.xml                   |    2 +-
 modules/slf4j/pom.xml                           |    2 +-
 modules/spark-2.10/pom.xml                      |    2 +-
 modules/spark/pom.xml                           |    2 +-
 .../org/apache/ignite/spark/IgniteContext.scala |   11 +-
 modules/spring/pom.xml                          |    4 +-
 .../ignite/internal/GridFactorySelfTest.java    |    2 +-
 modules/ssh/pom.xml                             |    2 +-
 modules/storm/README.txt                        |   37 +
 modules/storm/licenses/apache-2.0.txt           |  202 +++
 modules/storm/pom.xml                           |  102 ++
 .../ignite/stream/storm/StormStreamer.java      |  304 ++++
 .../ignite/stream/storm/package-info.java       |   22 +
 .../storm/IgniteStormStreamerSelfTestSuite.java |   38 +
 .../storm/StormIgniteStreamerSelfTest.java      |  184 +++
 .../ignite/stream/storm/TestStormSpout.java     |  141 ++
 .../storm/src/test/resources/example-ignite.xml |   71 +
 modules/tools/pom.xml                           |    2 +-
 modules/twitter/pom.xml                         |    2 +-
 modules/urideploy/pom.xml                       |    2 +-
 modules/visor-console-2.10/pom.xml              |    2 +-
 modules/visor-console/pom.xml                   |    2 +-
 .../commands/cache/VisorCacheCommand.scala      |   10 +-
 .../visor/commands/node/VisorNodeCommand.scala  |    1 +
 .../commands/tasks/VisorTasksCommand.scala      |    4 +-
 .../commands/top/VisorTopologyCommand.scala     |    5 +-
 .../scala/org/apache/ignite/visor/visor.scala   |    7 +-
 modules/visor-plugins/pom.xml                   |    2 +-
 modules/web/pom.xml                             |    4 +-
 modules/yardstick/.gitignore                    |    2 +
 .../config/benchmark-multicast.properties       |    7 +
 modules/yardstick/pom.xml                       |    2 +-
 .../org/apache/ignite/yardstick/IgniteNode.java |    3 +
 .../IgniteGetEntriesPutAllTxBenchmark.java      |   73 +
 .../yardstick/cache/IgniteGetTxBenchmark.java   |   30 +
 .../yardstick/cache/IgniteInvokeBenchmark.java  |   65 +
 .../cache/IgniteInvokeTxBenchmark.java          |   30 +
 .../cache/IgnitePutGetEntryBenchmark.java       |   47 +
 .../cache/IgnitePutGetEntryTxBenchmark.java     |   73 +
 modules/yarn/pom.xml                            |    2 +-
 .../apache/ignite/yarn/ApplicationMaster.java   |   12 +-
 .../apache/ignite/yarn/ClusterProperties.java   |  144 +-
 .../yarn/IgniteApplicationMasterSelfTest.java   |   52 +
 modules/zookeeper/pom.xml                       |    2 +-
 .../tcp/ipfinder/zk/ZookeeperIpFinderTest.java  |   80 +-
 parent/pom.xml                                  |  217 ++-
 pom.xml                                         |   20 +-
 871 files changed, 44558 insertions(+), 11002 deletions(-)
----------------------------------------------------------------------


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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9d66562/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9d66562/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9d66562/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9d66562/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9d66562/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 0d099f2,f11c2f8..01bb6e0
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@@ -89,6 -101,11 +100,7 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientReconnectTest;
  import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientTest;
  import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientTxReconnectTest;
+ import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryReconnectTest;
 -import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryAtomicSelfTest;
 -import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryLocalSelfTest;
 -import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryPartitionedSelfTest;
 -import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryReplicatedSelfTest;
  import org.apache.ignite.internal.processors.query.IgniteSqlSchemaIndexingTest;
  import org.apache.ignite.internal.processors.query.IgniteSqlSplitterSelfTest;
  import org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest;
@@@ -183,7 -209,16 +203,10 @@@ public class IgniteCacheQuerySelfTestSu
          suite.addTestSuite(CacheContinuousQueryFailoverAtomicReplicatedSelfTest.class);
          suite.addTestSuite(CacheContinuousQueryFailoverTxSelfTest.class);
          suite.addTestSuite(CacheContinuousQueryFailoverTxReplicatedSelfTest.class);
+         suite.addTestSuite(CacheContinuousQueryFailoverAtomicPrimaryWriteOrderOffheapTieredTest.class);
+         suite.addTestSuite(CacheContinuousQueryFailoverTxOffheapTieredTest.class);
+         suite.addTestSuite(CacheContinuousQueryRandomOperationsTest.class);
  
 -        // Reduce fields queries.
 -        suite.addTestSuite(GridCacheReduceFieldsQueryLocalSelfTest.class);
 -        suite.addTestSuite(GridCacheReduceFieldsQueryPartitionedSelfTest.class);
 -        suite.addTestSuite(GridCacheReduceFieldsQueryAtomicSelfTest.class);
 -        suite.addTestSuite(GridCacheReduceFieldsQueryReplicatedSelfTest.class);
 -
          suite.addTestSuite(GridCacheQueryIndexingDisabledSelfTest.class);
  
          suite.addTestSuite(GridCacheSwapScanQuerySelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9d66562/modules/spring/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/d9d66562/parent/pom.xml
----------------------------------------------------------------------


[17/50] [abbrv] ignite git commit: ignite-split2 - better debug + simpler test

Posted by sb...@apache.org.
ignite-split2 - better debug + simpler test


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

Branch: refs/heads/ignite-1232
Commit: ebfddeb033da6f9577364ffd4fdf410b3a89cf42
Parents: 4a46085
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Dec 8 04:08:30 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Dec 8 04:08:30 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/query/h2/opt/GridH2TreeIndex.java |  3 +++
 .../query/h2/twostep/msg/GridH2IndexRangeRequest.java     |  6 ++++++
 .../query/h2/twostep/msg/GridH2IndexRangeResponse.java    |  6 ++++++
 .../processors/query/IgniteSqlSplitterSelfTest.java       | 10 ++++++----
 4 files changed, 21 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ebfddeb0/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index 98b0b6a..9e5acf6 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -1379,6 +1379,9 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
 
             assert remainingRanges > 0;
 
+            if (log.isDebugEnabled())
+                log.debug("Starting stream: [node=" + node + ", req=" + req + "]");
+
             send(singletonList(node), req);
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebfddeb0/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java
index e72ff1e..2257c4c 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java
@@ -21,6 +21,7 @@ import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.UUID;
 import org.apache.ignite.internal.GridDirectCollection;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
@@ -194,4 +195,9 @@ public class GridH2IndexRangeRequest implements Message {
     @Override public byte fieldsCount() {
         return 4;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridH2IndexRangeRequest.class, this, "boundsSize", bounds == null ? null : bounds.size());
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebfddeb0/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java
index cb8c1a4..f75707f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java
@@ -21,6 +21,7 @@ import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.UUID;
 import org.apache.ignite.internal.GridDirectCollection;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
@@ -265,4 +266,9 @@ public class GridH2IndexRangeResponse implements Message {
     @Override public byte fieldsCount() {
         return 6;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridH2IndexRangeResponse.class, this, "rangesSize", ranges == null ? null : ranges.size());
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebfddeb0/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
index 879f343..c5ec8a9 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
@@ -218,7 +218,8 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
             awaitPartitionMapExchange();
 
             int key = 0;
-            for (int i = 0; i < 3000; i++) {
+
+            for (int i = 0; i < 30; i++) {
                 Organization o = new Organization();
 
                 o.name = "Org" + i;
@@ -227,11 +228,12 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
             }
 
             Random rnd = new GridRandom();
-            for (int i = 0; i < 15000; i++) {
+
+            for (int i = 0; i < 100; i++) {
                 Person p = new Person();
 
                 p.name = "Person" + i;
-                p.orgId = rnd.nextInt(3000);
+                p.orgId = rnd.nextInt(30);
 
                 c.put(key++, p);
             }
@@ -239,7 +241,7 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
             X.println("Plan : " + c.query(new SqlFieldsQuery("explain select count(*) from Person p, Organization o " +
                 "where p.orgId = o._key").setDistributedJoins(true)).getAll());
 
-            assertEquals(15000L, c.query(new SqlFieldsQuery("select count(*) from Person p, Organization o " +
+            assertEquals(100L, c.query(new SqlFieldsQuery("select count(*) from Person p, Organization o " +
                 "where p.orgId = o._key").setDistributedJoins(true)).getAll().get(0).get(0));
         }
         finally {


[39/50] [abbrv] ignite git commit: ignite-split2 - performance

Posted by sb...@apache.org.
ignite-split2 - performance


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

Branch: refs/heads/ignite-1232
Commit: 1115a794d785565b3d9be330595fa247cf156b7d
Parents: baba572
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Dec 28 00:26:23 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Dec 28 00:26:23 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/query/h2/opt/GridH2Table.java     | 10 +++++-----
 .../query/h2/twostep/GridReduceQueryExecutor.java         |  2 +-
 2 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1115a794/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
index afdf40f..ceb7922 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
@@ -275,9 +275,8 @@ public class GridH2Table extends TableBase {
 
         GridH2QueryContext qctx = GridH2QueryContext.get();
 
-        return qctx == null || // Outside of Ignite query context.
-            qctx.type() != MAP || // LOCAL and REDUCE queries.
-            (qctx.type() == MAP && !qctx.hasIndexSnapshots()); // Backward compatibility.
+        // On MAP queries with distributed joins we lock tables before the queries.
+        return qctx == null || qctx.type() != MAP || !qctx.hasIndexSnapshots();
     }
 
     /**
@@ -292,7 +291,7 @@ public class GridH2Table extends TableBase {
         Lock l;
 
         // Try to reuse existing snapshots outside of the lock.
-        for (long waitTime = 25;; waitTime *= 2) { // Increase wait time to avoid starvation.
+        for (long waitTime = 200;; waitTime *= 2) { // Increase wait time to avoid starvation.
             snapshots = actualSnapshot.get();
 
             if (snapshots != null) { // Reuse existing snapshot without locking.
@@ -395,7 +394,8 @@ public class GridH2Table extends TableBase {
 
             if (reuseExisting && s == null) { // Existing snapshot was invalidated before we were able to reserve it.
                 // Release already taken snapshots.
-                qctx.clearSnapshots();
+                if (qctx != null)
+                    qctx.clearSnapshots();
 
                 for (int j = 1; j < i; j++)
                     index(j).releaseSnapshot();

http://git-wip-us.apache.org/repos/asf/ignite/blob/1115a794/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index a5fca5d..a74681e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -618,7 +618,7 @@ public class GridReduceQueryExecutor {
                             .topologyVersion(topVer)
                             .pageSize(r.pageSize)
                             .caches(join(space, extraSpaces))
-                            .tables(qry.tables())
+                            .tables(distributedJoins ? qry.tables() : null)
                             .partitions(convert(partsMap))
                             .queries(mapQrys)
                             .flags(distributedJoins ? GridH2QueryRequest.FLAG_DISTRIBUTED_JOINS : 0),


[38/50] [abbrv] ignite git commit: ignite-split2 - test fix

Posted by sb...@apache.org.
ignite-split2 - test fix


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

Branch: refs/heads/ignite-1232
Commit: baba57287f962551dd87234fe80c2ddf3e5ca2cc
Parents: 2bdbd7e
Author: S.Vladykin <sv...@gridgain.com>
Authored: Sat Dec 26 12:59:32 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Sat Dec 26 12:59:32 2015 +0300

----------------------------------------------------------------------
 .../offheap/unsafe/GridOffHeapSnapTreeMap.java  |  5 +++--
 .../query/h2/opt/GridH2QueryContext.java        | 22 ++++++++++++++------
 .../processors/query/h2/opt/GridH2Table.java    |  2 ++
 .../query/h2/opt/GridH2TreeIndex.java           |  2 ++
 4 files changed, 23 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/baba5728/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridOffHeapSnapTreeMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridOffHeapSnapTreeMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridOffHeapSnapTreeMap.java
index 9b136d2..7bde651 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridOffHeapSnapTreeMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridOffHeapSnapTreeMap.java
@@ -1048,7 +1048,7 @@ public class GridOffHeapSnapTreeMap<K extends GridOffHeapSmartPointer,V extends
     private volatile StoppableRecycleQueue recycleBin;
 
     /** */
-    private final AtomicInteger reservations = new AtomicInteger();
+    private AtomicInteger reservations;
 
     /** */
     private volatile boolean closing;
@@ -1125,7 +1125,7 @@ public class GridOffHeapSnapTreeMap<K extends GridOffHeapSmartPointer,V extends
     @Override public void close() {
         closing = true;
 
-        if (reservations.compareAndSet(0, -1))
+        if (reservations == null || reservations.compareAndSet(0, -1))
             doClose();
     }
 
@@ -1192,6 +1192,7 @@ public class GridOffHeapSnapTreeMap<K extends GridOffHeapSmartPointer,V extends
         copy.holderRef = rootHolder(holderRef);
         markShared(root());
 
+        copy.reservations = new AtomicInteger();
         copy.size = new AtomicInteger(size());
         copy.recycleBin = new StoppableRecycleQueue();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/baba5728/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
index d5fc41d..223dad6 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
@@ -245,6 +245,21 @@ public class GridH2QueryContext {
     }
 
     /**
+     * Clear taken snapshots.
+     */
+    public void clearSnapshots() {
+        if (F.isEmpty(snapshots))
+            return;
+
+        for (Object snapshot : snapshots.values()) {
+            if (snapshot instanceof GridReservable)
+                ((GridReservable)snapshot).release();
+        }
+
+        snapshots = null;
+    }
+
+    /**
      * @param idxId Index ID.
      * @return Index snapshot or {@code null} if none.
      */
@@ -389,12 +404,7 @@ public class GridH2QueryContext {
 
         assert x.key.equals(key);
 
-        if (!F.isEmpty(x.snapshots)) {
-            for (Object snapshot : x.snapshots.values()) {
-                if (snapshot instanceof GridReservable)
-                    ((GridReservable)snapshot).release();
-            }
-        }
+        x.clearSnapshots();
 
         List<GridReservable> r = x.reservations;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/baba5728/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
index ac21807..afdf40f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
@@ -395,6 +395,8 @@ public class GridH2Table extends TableBase {
 
             if (reuseExisting && s == null) { // Existing snapshot was invalidated before we were able to reserve it.
                 // Release already taken snapshots.
+                qctx.clearSnapshots();
+
                 for (int j = 1; j < i; j++)
                     index(j).releaseSnapshot();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/baba5728/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index 95efc8e..1d4efff 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -376,6 +376,8 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
 
     /** {@inheritDoc} */
     @Override protected Object doTakeSnapshot() {
+        assert snapshotEnabled;
+
         return tree instanceof SnapTreeMap ?
             ((SnapTreeMap)tree).clone() :
             ((GridOffHeapSnapTreeMap)tree).clone();


[05/50] [abbrv] ignite git commit: ignite-split2 - minor fixes

Posted by sb...@apache.org.
ignite-split2 - minor fixes


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

Branch: refs/heads/ignite-1232
Commit: 5c4f8b22dd230de03e7552f87000fb2acc804884
Parents: 5a0bf09
Author: S.Vladykin <sv...@gridgain.com>
Authored: Sat Dec 5 18:31:50 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Sat Dec 5 18:31:50 2015 +0300

----------------------------------------------------------------------
 .../query/h2/opt/GridH2Collocation.java         | 37 ++++++++------------
 1 file changed, 14 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5c4f8b22/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java
index 13af99b..10cd0fe 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java
@@ -82,20 +82,6 @@ public final class GridH2Collocation {
     }
 
     /**
-     * @param select Select.
-     */
-    public void select(Select select) {
-        this.select = select;
-    }
-
-    /**
-     * @return Select.
-     */
-    public Select select() {
-        return select;
-    }
-
-    /**
      * @return List of unions.
      */
     public List<GridH2Collocation> unions() {
@@ -115,9 +101,17 @@ public final class GridH2Collocation {
      */
     private boolean childFilters(TableFilter[] childFilters) {
         assert childFilters != null;
-        assert select == childFilters[0].getSelect();
 
-        if (Arrays.equals(this.childFilters, childFilters))
+        Select select = childFilters[0].getSelect();
+
+        assert this.select == null || this.select == select;
+
+        if (this.select == null) {
+            this.select = select;
+
+            assert this.childFilters == null;
+        }
+        else if (Arrays.equals(this.childFilters, childFilters))
             return false;
 
         childsOrderFinalized = false;
@@ -545,17 +539,15 @@ public final class GridH2Collocation {
             if (c == null) {
                 c = new GridH2Collocation(null, -1);
 
-                c.select(filters[0].getSelect());
-
                 qctx.queryCollocation(c);
             }
         }
 
-        // Handle union. We have to rely on fact that select will be the same on uppermost select.
-        // For sub-queries we will drop collocation models, so that they will be recalculated anyways.
         Select select = filters[0].getSelect();
 
-        if (c.select() != select) {
+        // Handle union. We have to rely on fact that select will be the same on uppermost select.
+        // For sub-queries we will drop collocation models, so that they will be recalculated anyways.
+        if (c.select != null && c.select != select) {
             List<GridH2Collocation> unions = c.unions();
 
             int i = 1;
@@ -570,7 +562,7 @@ public final class GridH2Collocation {
                 for (; i < unions.size(); i++) {
                     GridH2Collocation u = unions.get(i);
 
-                    if (u.select() == select) {
+                    if (u.select == select) {
                         c = u;
 
                         break;
@@ -583,7 +575,6 @@ public final class GridH2Collocation {
 
                 unions.add(c);
 
-                c.select(select);
                 c.unions(unions);
             }
         }


[46/50] [abbrv] ignite git commit: ignite-split2 - separate pool for index requests + parallel execution of local query in case of distributed join

Posted by sb...@apache.org.
ignite-split2 - separate pool for index requests + parallel execution of local query in case of distributed join


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

Branch: refs/heads/ignite-1232
Commit: c5dd4bc99f487b184e246fd483ce21822226a7ae
Parents: 9483473
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Jan 11 16:44:16 2016 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Jan 11 16:44:16 2016 +0300

----------------------------------------------------------------------
 .../managers/communication/GridIoManager.java   | 16 +++++++
 .../managers/communication/GridIoPolicy.java    |  3 ++
 .../closure/GridClosureProcessor.java           |  2 +-
 .../processors/query/h2/IgniteH2Indexing.java   | 50 ++++++++++++++++++--
 .../query/h2/opt/GridH2TreeIndex.java           |  9 +++-
 .../h2/twostep/GridReduceQueryExecutor.java     | 15 +++---
 6 files changed, 83 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c5dd4bc9/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index ec2d797..32c96ca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -87,6 +87,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.internal.GridTopic.TOPIC_COMM_USER;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.AFFINITY_POOL;
+import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IDX_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IGFS_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.MANAGEMENT_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.MARSH_CACHE_POOL;
@@ -147,6 +148,9 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
     /** IGFS pool. */
     private ExecutorService igfsPool;
 
+    /** Index pool. */
+    private ExecutorService idxPool;
+
     /** Discovery listener. */
     private GridLocalEventListener discoLsnr;
 
@@ -254,6 +258,13 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
             0,
             new LinkedBlockingQueue<Runnable>());
 
+        if (IgniteComponentType.INDEXING.inClassPath()) {
+            int cpus = Runtime.getRuntime().availableProcessors();
+
+            idxPool = new IgniteThreadPoolExecutor("idx", ctx.gridName(),
+                cpus, cpus * 2, 3000L, new LinkedBlockingQueue<Runnable>(1000));
+        }
+
         getSpi().setListener(commLsnr = new CommunicationListener<Serializable>() {
             @Override public void onMessage(UUID nodeId, Serializable msg, IgniteRunnable msgC) {
                 try {
@@ -701,6 +712,11 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
 
                 return igfsPool;
 
+            case IDX_POOL:
+                assert idxPool != null : "Indexing pool is not configured.";
+
+                return idxPool;
+
             default: {
                 assert plc >= 0 : "Negative policy: " + plc;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c5dd4bc9/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java
index a417857..70a7354 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java
@@ -46,6 +46,9 @@ public class GridIoPolicy {
     /** IGFS pool. */
     public static final byte IGFS_POOL = 7;
 
+    /** Pool for handling distributed index range requests. */
+    public static final byte IDX_POOL = 8;
+
     /**
      * Defines the range of reserved pools that are not available for plugins.
      * @param key The key.

http://git-wip-us.apache.org/repos/asf/ignite/blob/c5dd4bc9/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
index c53cb8b..c4a915e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
@@ -744,7 +744,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @return Future.
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
-    private IgniteInternalFuture<?> runLocal(@Nullable final Runnable c, byte plc) throws IgniteCheckedException {
+    public IgniteInternalFuture<?> runLocal(@Nullable final Runnable c, byte plc) throws IgniteCheckedException {
         if (c == null)
             return new GridFinishedFuture();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c5dd4bc9/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 58026ca..f6c312e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -101,6 +101,7 @@ import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
 import org.apache.ignite.internal.util.GridEmptyCloseableIterator;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.lang.GridCloseableIterator;
+import org.apache.ignite.internal.util.lang.GridPlainRunnable;
 import org.apache.ignite.internal.util.lang.IgniteInClosure2X;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeGuard;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
@@ -113,6 +114,7 @@ import org.apache.ignite.lang.IgniteBiClosure;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.jdk.JdkMarshaller;
 import org.apache.ignite.plugin.extensions.communication.Message;
@@ -314,6 +316,18 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     private final GridBoundedConcurrentLinkedHashMap<TwoStepCachedQueryKey, TwoStepCachedQuery> twoStepCache =
         new GridBoundedConcurrentLinkedHashMap<>(TWO_STEP_QRY_CACHE_SIZE);
 
+    /** */
+    private final IgniteInClosure<? super IgniteInternalFuture<?>> logger = new IgniteInClosure<IgniteInternalFuture<?>>() {
+        @Override public void apply(IgniteInternalFuture<?> fut) {
+            try {
+                fut.get();
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, e.getMessage(), e);
+            }
+        }
+    };
+
     /**
      * @return Kernal context.
      */
@@ -1575,6 +1589,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @param msg Message.
      * @param specialize Optional closure to specialize message for each node.
      * @param locNodeHandler Handler for local node.
+     * @param plc Policy identifying the executor service which will process message.
+     * @param runLocParallel Run local handler in parallel thread.
      * @return {@code true} If all messages sent successfully.
      */
     public boolean send(
@@ -1582,7 +1598,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         Collection<ClusterNode> nodes,
         Message msg,
         @Nullable IgniteBiClosure<ClusterNode, Message, Message> specialize,
-        @Nullable IgniteInClosure2X<ClusterNode, Message> locNodeHandler
+        @Nullable final IgniteInClosure2X<ClusterNode, Message> locNodeHandler,
+        byte plc,
+        boolean runLocParallel
     ) {
         boolean ok = true;
 
@@ -1606,7 +1624,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                         ((GridCacheQueryMarshallable)msg).marshall(marshaller);
                 }
 
-                ctx.io().send(node, topic, msg, GridReduceQueryExecutor.QUERY_POOL);
+                ctx.io().send(node, topic, msg, plc);
             }
             catch (IgniteCheckedException e) {
                 ok = false;
@@ -1616,8 +1634,32 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             }
         }
 
-        if (locNode != null) // Local node goes the last to allow parallel execution.
-            locNodeHandler.apply(locNode, specialize == null ? msg : specialize.apply(locNode, msg));
+        // Local node goes the last to allow parallel execution.
+        if (locNode != null) {
+            if (specialize != null)
+                msg = specialize.apply(locNode, msg);
+
+            if (runLocParallel) {
+                final ClusterNode finalLocNode = locNode;
+                final Message finalMsg = msg;
+
+                try {
+                    // We prefer runLocal to runLocalSafe, because the latter can produce deadlock here.
+                    ctx.closure().runLocal(new GridPlainRunnable() {
+                        @Override public void run() {
+                            locNodeHandler.apply(finalLocNode, finalMsg);
+                        }
+                    }, plc).listen(logger);
+                }
+                catch (IgniteCheckedException e) {
+                    ok = false;
+
+                    U.error(log, "Failed to execute query locally.", e);
+                }
+            }
+            else
+                locNodeHandler.apply(locNode, msg);
+        }
 
         return ok;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c5dd4bc9/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index f624292..fd2f174 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -41,6 +41,7 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.GridTopic;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2IndexRangeRequest;
@@ -237,7 +238,8 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
      * @param msg Message.
      */
     private void send(Collection<ClusterNode> nodes, Message msg) {
-        if (!getTable().rowDescriptor().indexing().send(msgTopic, nodes, msg, null, locNodeHandler))
+        if (!getTable().rowDescriptor().indexing().send(msgTopic, nodes, msg, null, locNodeHandler,
+            GridIoPolicy.IDX_POOL, false))
             throw new GridH2RetryException("Failed to send message to nodes: " + nodes + ".");
     }
 
@@ -1410,6 +1412,8 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
         private GridH2IndexRangeResponse awaitForResponse() {
             assert remainingRanges > 0;
 
+            final long start = U.currentTimeMillis();
+
             for (int attempt = 0;; attempt++) {
                 if (qctx.isCleared())
                     throw new GridH2RetryException("Query is cancelled.");
@@ -1452,6 +1456,9 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
                             if (req == null || req.bounds() == null) // We have already received the first response.
                                 throw new GridH2RetryException("Failure on remote node.");
 
+                            if (U.currentTimeMillis() - start > 30_000)
+                                throw new GridH2RetryException("Timeout.");
+
                             try {
                                 U.sleep(20 * attempt);
                             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c5dd4bc9/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 409e26f..3151677 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -627,8 +627,9 @@ public class GridReduceQueryExecutor {
                             .partitions(convert(partsMap))
                             .queries(mapQrys)
                             .flags(distributedJoins ? GridH2QueryRequest.FLAG_DISTRIBUTED_JOINS : 0),
-                    oldStyle && partsMap != null ? new ExplicitPartitionsSpecializer(partsMap) : null))
-                {
+                    oldStyle && partsMap != null ? new ExplicitPartitionsSpecializer(partsMap) : null,
+                    distributedJoins)
+                ) {
                     awaitAllReplies(r, nodes);
 
                     Object state = r.state.get();
@@ -710,7 +711,7 @@ public class GridReduceQueryExecutor {
                 }
 
                 if (retry) {
-                    send(nodes, new GridQueryCancelRequest(qryReqId), null);
+                    send(nodes, new GridQueryCancelRequest(qryReqId), null, false);
 
                     if (Thread.currentThread().isInterrupted())
                         throw new IgniteInterruptedCheckedException("Query was interrupted.");
@@ -725,7 +726,7 @@ public class GridReduceQueryExecutor {
                         super.close();
 
                         if (distributedJoins || !allIndexesFetched(r.idxs))
-                            send(finalNodes, new GridQueryCancelRequest(qryReqId), null);
+                            send(finalNodes, new GridQueryCancelRequest(qryReqId), null, false);
                     }
                 };
             }
@@ -1121,17 +1122,19 @@ public class GridReduceQueryExecutor {
      * @param nodes Nodes.
      * @param msg Message.
      * @param specialize Optional closure to specialize message for each node.
+     * @param runLocParallel Run local handler in parallel thread.
      * @return {@code true} If all messages sent successfully.
      */
     private boolean send(
         Collection<ClusterNode> nodes,
         Message msg,
-        @Nullable IgniteBiClosure<ClusterNode, Message, Message> specialize
+        @Nullable IgniteBiClosure<ClusterNode, Message, Message> specialize,
+        boolean runLocParallel
     ) {
         if (log.isDebugEnabled())
             log.debug("Sending: [msg=" + msg + ", nodes=" + nodes + ", specialize=" + specialize + "]");
 
-        return h2.send(GridTopic.TOPIC_QUERY, nodes, msg, specialize, locNodeHandler);
+        return h2.send(GridTopic.TOPIC_QUERY, nodes, msg, specialize, locNodeHandler, QUERY_POOL, runLocParallel);
     }
 
     /**


[19/50] [abbrv] ignite git commit: ignite-split2 - compatibility

Posted by sb...@apache.org.
ignite-split2 - compatibility


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

Branch: refs/heads/ignite-1232
Commit: 281fe7cf4725b31ac88b6315d487a8f2f0fb18d2
Parents: 1113c6f
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Dec 8 04:26:03 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Dec 8 04:26:03 2015 +0300

----------------------------------------------------------------------
 .../query/h2/twostep/msg/GridH2ValueMessage.java    | 16 +---------------
 1 file changed, 1 insertion(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/281fe7cf/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java
index 3a9f43a..ee8856b 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessage.java
@@ -40,25 +40,11 @@ public abstract class GridH2ValueMessage implements Message {
 
     /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
-        writer.setBuffer(buf);
-
-        if (!writer.isHeaderWritten()) {
-            if (!writer.writeHeader(directType(), fieldsCount()))
-                return false;
-
-            writer.onHeaderWritten();
-        }
-
         return true;
     }
 
     /** {@inheritDoc} */
     @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
-        reader.setBuffer(buf);
-
-        if (!reader.beforeMessageRead())
-            return false;
-
-        return reader.afterMessageRead(GridH2ValueMessage.class);
+        return true;
     }
 }


[02/50] [abbrv] ignite git commit: ignite-split2 - refactor

Posted by sb...@apache.org.
ignite-split2 - refactor


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

Branch: refs/heads/ignite-1232
Commit: f5c63701da73d2b3405d1d24ebffc2e6eabfca12
Parents: 4e16d0d
Author: S.Vladykin <sv...@gridgain.com>
Authored: Fri Dec 4 07:53:25 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Fri Dec 4 07:53:25 2015 +0300

----------------------------------------------------------------------
 .../query/h2/opt/GridH2Collocation.java         | 736 +++++++++++++++++++
 .../query/h2/opt/GridH2IndexBase.java           | 371 +---------
 .../query/h2/opt/GridH2QueryContext.java        |  20 +-
 .../processors/query/h2/opt/GridH2Table.java    |   9 +-
 .../h2/opt/GridH2TableFilterCollocation.java    |  64 --
 .../query/h2/opt/GridH2TreeIndex.java           |   2 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |   2 +-
 7 files changed, 767 insertions(+), 437 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f5c63701/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java
new file mode 100644
index 0000000..fd72b2b
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java
@@ -0,0 +1,736 @@
+/*
+ * 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.query.h2.opt;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.h2.command.dml.Query;
+import org.h2.command.dml.Select;
+import org.h2.command.dml.SelectUnion;
+import org.h2.expression.Expression;
+import org.h2.expression.ExpressionColumn;
+import org.h2.index.IndexCondition;
+import org.h2.index.ViewIndex;
+import org.h2.table.Column;
+import org.h2.table.SubQueryInfo;
+import org.h2.table.Table;
+import org.h2.table.TableFilter;
+
+/**
+ * Collocation model for a query.
+ */
+public class GridH2Collocation {
+    /** */
+    public static final int MULTIPLIER_COLLOCATED = 1;
+
+    /** */
+    private static final int MULTIPLIER_UNICAST = 20;
+
+    /** */
+    private static final int MULTIPLIER_BROADCAST = 80;
+
+    /** */
+    private final GridH2Collocation upper;
+
+    /** */
+    private final int filter;
+
+    /** */
+    private int multiplier;
+
+    /** */
+    private Type type;
+
+    /** */
+    private GridH2Collocation[] children;
+
+    /** */
+    private TableFilter[] childFilters;
+
+    /** */
+    private boolean childsOrderFinalized;
+
+    /** */
+    private List<GridH2Collocation> unions;
+
+    /** */
+    private Select select;
+
+    /**
+     * @param upper Upper.
+     * @param filter Filter.
+     */
+    public GridH2Collocation(GridH2Collocation upper, int filter) {
+        this.upper = upper;
+        this.filter = filter;
+    }
+
+    /**
+     * @param select Select.
+     */
+    public void select(Select select) {
+        this.select = select;
+    }
+
+    /**
+     * @return Select.
+     */
+    public Select select() {
+        return select;
+    }
+
+    /**
+     * @return List of unions.
+     */
+    public List<GridH2Collocation> unions() {
+        return unions;
+    }
+
+    /**
+     * @param unions Unions.
+     */
+    public void unions(List<GridH2Collocation> unions) {
+        this.unions = unions;
+    }
+
+    /**
+     * @param childFilters New child filters.
+     * @return {@code true} If child filters were updated.
+     */
+    public boolean childFilters(TableFilter[] childFilters) {
+        assert childFilters != null;
+        assert select == childFilters[0].getSelect();
+
+        if (Arrays.equals(this.childFilters, childFilters))
+            return false;
+
+        childsOrderFinalized = false;
+
+        if (this.childFilters == null) {
+            // We have to clone because H2 reuses array and reorders elements.
+            this.childFilters = childFilters.clone();
+
+            children = new GridH2Collocation[childFilters.length];
+        }
+        else {
+            assert this.childFilters.length == childFilters.length;
+
+            // We have to copy because H2 reuses array and reorders elements.
+            System.arraycopy(childFilters, 0, this.childFilters, 0, childFilters.length);
+
+            Arrays.fill(children, null);
+        }
+
+        reset();
+
+        return true;
+    }
+
+    /**
+     * Reset current collocation model and all the children, but do not touch union.
+     */
+    private void reset() {
+        type = null;
+        multiplier = 0;
+    }
+
+    /**
+     * @param i Index.
+     * @param f Table filter.
+     * @return {@code true} If the child is not a table or view.
+     */
+    private boolean isNotTableOrViewChild(int i, TableFilter f) {
+        if (f == null)
+            f = childFilters[i];
+
+        Table t = f.getTable();
+
+        return !t.isView() && !(t instanceof GridH2Table);
+    }
+
+    /**
+     * Do the needed calculations.
+     */
+    private void calculate() {
+        if (type != null)
+            return;
+
+        if (childFilters != null) {
+            // We are at sub-query.
+            boolean collocated = true;
+            boolean partitioned = false;
+            int maxMultiplier = 0;
+
+            for (int i = 0; i < childFilters.length; i++) {
+                GridH2Collocation c = child(i);
+
+                if (c == null) {
+                    assert isNotTableOrViewChild(i, null);
+
+                    continue;
+                }
+
+                Type t = c.type(true);
+
+                if (!t.isCollocated()) {
+                    collocated = false;
+
+                    int m = c.multiplier(true);
+
+                    if (m > maxMultiplier)
+                        maxMultiplier = m;
+                }
+
+                if (t.isPartitioned())
+                    partitioned = true;
+            }
+
+            type = Type.of(partitioned, collocated);
+            multiplier = type.isCollocated() ? MULTIPLIER_COLLOCATED : maxMultiplier;
+        }
+        else {
+            assert upper != null;
+
+            // We are at table instance.
+            GridH2Table tbl = (GridH2Table)upper.childFilters[filter].getTable();
+
+            // Only partitioned tables will do distributed joins.
+            if (!tbl.isPartitioned()) {
+                type = Type.REPLICATED;
+                multiplier = MULTIPLIER_COLLOCATED;
+
+                return;
+            }
+
+            // If we are the first partitioned table in a join, then we are "base" for all the rest partitioned tables
+            // which will need to get remote result (if there is no affinity condition). Since this query is broadcasted
+            // to all the affinity nodes the "base" does not need to get remote results.
+            if (!upper.findPartitionedTableBefore(filter)) {
+                type = Type.PARTITIONED_COLLOCATED;
+                multiplier = MULTIPLIER_COLLOCATED;
+
+                return;
+            }
+
+            // It is enough to make sure that our previous join by affinity key is collocated, then we are
+            // collocated. If we at least have affinity key condition, then we do unicast which is cheaper.
+            switch (upper.joinedWithCollocated(filter)) {
+                case JOINED_WITH_COLLOCATED:
+                    type = Type.PARTITIONED_COLLOCATED;
+                    multiplier = MULTIPLIER_COLLOCATED;
+
+                    break;
+
+                case HAS_AFFINITY_CONDITION:
+                    type = Type.PARTITIONED_NOT_COLLOCATED;
+                    multiplier = MULTIPLIER_UNICAST;
+
+                    break;
+
+                case NONE:
+                    type = Type.PARTITIONED_NOT_COLLOCATED;
+                    multiplier = MULTIPLIER_BROADCAST;
+
+                    break;
+            }
+        }
+    }
+
+    /**
+     * @param f Current filter.
+     * @return {@code true} If partitioned table was found.
+     */
+    private boolean findPartitionedTableBefore(int f) {
+        for (int i = 0; i < f; i++) {
+            GridH2Collocation c = child(i);
+
+            assert c != null || isNotTableOrViewChild(i, null);
+
+            // The `c` can be null if it is not a GridH2Table and not a sub-query,
+            // it is a some kind of function table or anything else that considered replicated.
+            if (c != null && c.type(true).isPartitioned())
+                return true;
+        }
+
+        // We have to search globally in upper queries as well.
+        return upper != null && upper.findPartitionedTableBefore(filter);
+    }
+
+    /**
+     * @param f Filter.
+     * @return Affinity join type.
+     */
+    private Affinity joinedWithCollocated(int f) {
+        TableFilter tf = childFilters[f];
+
+        ArrayList<IndexCondition> idxConditions = tf.getIndexConditions();
+
+        int affColId = ((GridH2Table)tf.getTable()).getAffinityKeyColumnId();
+
+        boolean affKeyConditionFound = false;
+
+        for (int i = 0; i < idxConditions.size(); i++) {
+            IndexCondition c = idxConditions.get(i);
+
+            if (c.getCompareType() == IndexCondition.EQUALITY &&
+                c.getColumn().getColumnId() == affColId && c.isEvaluatable()) {
+                affKeyConditionFound = true;
+
+                Expression exp = c.getExpression();
+
+                exp = exp.getNonAliasExpression();
+
+                if (exp instanceof ExpressionColumn) {
+                    ExpressionColumn expCol = (ExpressionColumn)exp;
+
+                    // This is one of our previous joins.
+                    TableFilter prevJoin = expCol.getTableFilter();
+
+                    if (prevJoin != null) {
+                        GridH2Collocation co = children[indexOf(prevJoin)];
+
+                        assert co != null || isNotTableOrViewChild(-1, prevJoin);
+
+                        if (co != null) {
+                            Type t = co.type(true);
+
+                            if (t.isPartitioned() && t.isCollocated() && isAffinityColumn(prevJoin, expCol))
+                                return Affinity.JOINED_WITH_COLLOCATED;
+                        }
+                    }
+                }
+            }
+        }
+
+        return affKeyConditionFound ? Affinity.HAS_AFFINITY_CONDITION : Affinity.NONE;
+    }
+
+    /**
+     * @param f Table filter.
+     * @return Index.
+     */
+    private int indexOf(TableFilter f) {
+        for (int i = 0; i < childFilters.length; i++) {
+            if (childFilters[i] == f)
+                return i;
+        }
+
+        throw new IllegalStateException();
+    }
+
+    /**
+     * @param f Table filter.
+     * @param expCol Expression column.
+     * @return {@code true} It it is an affinity column.
+     */
+    private static boolean isAffinityColumn(TableFilter f, ExpressionColumn expCol) {
+        Column col = expCol.getColumn();
+
+        if (col == null)
+            return false;
+
+        Table t = col.getTable();
+
+        if (t.isView()) {
+            Query qry = ((ViewIndex)f.getIndex()).getQuery();
+
+            return isAffinityColumn(qry, expCol);
+        }
+
+        return t instanceof GridH2Table &&
+            col.getColumnId() == ((GridH2Table)t).getAffinityKeyColumnId();
+    }
+
+    /**
+     * @param qry Query.
+     * @param expCol Expression column.
+     * @return {@code true} It it is an affinity column.
+     */
+    private static boolean isAffinityColumn(Query qry, ExpressionColumn expCol) {
+        if (qry.isUnion()) {
+            SelectUnion union = (SelectUnion)qry;
+
+            return isAffinityColumn(union.getLeft(), expCol) && isAffinityColumn(union.getRight(), expCol);
+        }
+
+        Expression exp = qry.getExpressions().get(expCol.getColumn().getColumnId()).getNonAliasExpression();
+
+        if (exp instanceof ExpressionColumn) {
+            expCol = (ExpressionColumn)exp;
+
+            return isAffinityColumn(expCol.getTableFilter(), expCol);
+        }
+
+        return false;
+    }
+
+    /**
+     * Sets table filters to the final state of query.
+     *
+     * @return {@code false} if nothing was actually done here.
+     */
+    private boolean finalizeChildFiltersOrder() {
+        if (childFilters == null || childsOrderFinalized)
+            return false;
+
+        int i = 0;
+
+        // Collect table filters in final order after optimization.
+        for (TableFilter f = select.getTopTableFilter(); f != null; f = f.getJoin()) {
+            childFilters[i] = f;
+
+            if (f.getTable().isView())
+                children[i].finalizeChildFiltersOrder();
+
+            i++;
+        }
+
+        assert i == childFilters.length;
+
+        reset();
+
+        childsOrderFinalized = true;
+
+        return true;
+    }
+
+    /**
+     * @return Multiplier.
+     */
+    public int calculateMultiplier() {
+        if (childFilters != null && !childsOrderFinalized) {
+            // We have to set all sub-queries structure to the final one we will have in query.
+            boolean needReset = false;
+
+            for (int i = 0; i < childFilters.length; i++) {
+                if (childFilters[i].getTable().isView() && children[i].finalizeChildFiltersOrder())
+                    needReset = true;
+            }
+
+            if (needReset)
+                reset();
+
+            childsOrderFinalized = true;
+        }
+
+        // We don't need multiplier for union here because it will be summarized in H2.
+        return multiplier(false);
+    }
+
+    /**
+     * @param withUnion With respect to union.
+     * @return Multiplier.
+     */
+    private int multiplier(boolean withUnion) {
+        calculate();
+
+        assert multiplier != 0;
+
+        if (withUnion && unions != null) {
+            int maxMultiplier = unions.get(0).multiplier(false);
+
+            for (int i = 1; i < unions.size(); i++) {
+                int m = unions.get(i).multiplier(false);
+
+                if (m > maxMultiplier)
+                    maxMultiplier = m;
+            }
+
+            return maxMultiplier;
+        }
+
+        return multiplier;
+    }
+
+    /**
+     * @param withUnion With respect to union.
+     * @return Type.
+     */
+    private Type type(boolean withUnion) {
+        calculate();
+
+        assert type != null;
+
+        if (withUnion && unions != null) {
+            Type left = unions.get(0).type(false);
+
+            for (int i = 1; i < unions.size(); i++) {
+                Type right = unions.get(i).type(false);
+
+                if (!left.isCollocated() || !right.isCollocated()) {
+                    left = Type.PARTITIONED_NOT_COLLOCATED;
+
+                    break;
+                }
+                else if (!left.isPartitioned() && !right.isPartitioned())
+                    left = Type.REPLICATED;
+                else
+                    left = Type.PARTITIONED_COLLOCATED;
+            }
+
+            return left;
+        }
+
+        return type;
+    }
+
+    /**
+     * @param idx Index.
+     * @param child Child collocation.
+     */
+    public void child(int idx, GridH2Collocation child) {
+        assert child(idx) == null;
+
+        children[idx] = child;
+    }
+
+    /**
+     * @param idx Index.
+     * @return Child collocation.
+     */
+    public GridH2Collocation child(int idx) {
+        return children[idx];
+    }
+
+    /**
+     * @return Upper collocation.
+     */
+    public GridH2Collocation upper() {
+        return upper;
+    }
+
+    /**
+     * @return Filter.
+     */
+    public int filter() {
+        return filter;
+    }
+
+    /**
+     * @param qctx Query context.
+     * @param info Sub-query info.
+     * @param filters Filters.
+     * @param filter Filter.
+     * @return Collocation.
+     */
+    public static GridH2Collocation buildCollocationModel(GridH2QueryContext qctx, SubQueryInfo info,
+        TableFilter[] filters, int filter) {
+        GridH2Collocation c;
+
+        if (info != null) {
+            // Go up until we reach the root query.
+            c = buildCollocationModel(qctx, info.getUpper(), info.getFilters(), info.getFilter());
+        }
+        else {
+            // We are at the root query.
+            c = qctx.queryCollocation();
+
+            if (c == null) {
+                c = new GridH2Collocation(null, -1);
+
+                c.select(filters[0].getSelect());
+
+                qctx.queryCollocation(c);
+            }
+        }
+
+        // Handle union. We have to rely on fact that select will be the same on uppermost select.
+        // For sub-queries we will drop collocation models, so that they will be recalculated anyways.
+        Select select = filters[0].getSelect();
+
+        if (c.select() != select) {
+            List<GridH2Collocation> unions = c.unions();
+
+            int i = 1;
+
+            if (unions == null) {
+                unions = new ArrayList<>();
+
+                unions.add(c);
+                c.unions(unions);
+            }
+            else {
+                for (; i < unions.size(); i++) {
+                    GridH2Collocation u = unions.get(i);
+
+                    if (u.select() == select) {
+                        c = u;
+
+                        break;
+                    }
+                }
+            }
+
+            if (i == unions.size()) {
+                c = new GridH2Collocation(c.upper(), c.filter());
+
+                unions.add(c);
+
+                c.select(select);
+                c.unions(unions);
+            }
+        }
+
+        c.childFilters(filters);
+
+        GridH2Collocation child = c.child(filter);
+
+        if (child == null) {
+            child = new GridH2Collocation(c, filter);
+
+            c.child(filter, child);
+        }
+
+        return child;
+    }
+
+    /**
+     * @param qry Query.
+     * @return {@code true} If the query is collocated.
+     */
+    public static boolean isCollocated(Query qry) {
+        return buildCollocationModel(null, -1, qry, null).type(true).isCollocated();
+    }
+
+    /**
+     * @param upper Upper.
+     * @param filter Filter.
+     * @param qry Query.
+     * @param unions Unions.
+     * @return Built model.
+     */
+    private static GridH2Collocation buildCollocationModel(GridH2Collocation upper, int filter, Query qry,
+        List<GridH2Collocation> unions) {
+        if (qry.isUnion()) {
+            if (unions == null)
+                unions = new ArrayList<>();
+
+            SelectUnion union = (SelectUnion)qry;
+
+            GridH2Collocation a = buildCollocationModel(upper, filter, union.getLeft(), unions);
+            GridH2Collocation b = buildCollocationModel(upper, filter, union.getRight(), unions);
+
+            return a == null ? b : a;
+        }
+
+        Select select = (Select)qry;
+
+        List<TableFilter> list = new ArrayList<>();
+
+        for (TableFilter f = select.getTopTableFilter(); f != null; f = f.getJoin())
+            list.add(f);
+
+        TableFilter[] filters = list.toArray(new TableFilter[list.size()]);
+
+        GridH2Collocation c = new GridH2Collocation(upper, filter);
+
+        if (unions != null) {
+            unions.add(c);
+
+            c.unions(unions);
+        }
+
+        c.childFilters(filters);
+
+        if (upper != null)
+            upper.child(filter, c);
+
+        for (int i = 0; i < filters.length; i++) {
+            TableFilter f = filters[i];
+
+            if (f.getTable().isView())
+                c.child(i, buildCollocationModel(c, i, ((ViewIndex)f.getIndex()).getQuery(), null));
+            else if (f.getTable() instanceof GridH2Table)
+                c.child(i, new GridH2Collocation(c, i));
+        }
+
+        return upper == null ? c : null;
+    }
+
+    /**
+     * Collocation type.
+     */
+    private enum Type {
+        /** */
+        PARTITIONED_COLLOCATED(true, true),
+
+        /** */
+        PARTITIONED_NOT_COLLOCATED(true, false),
+
+        /** */
+        REPLICATED(false, true);
+
+        /** */
+        private final boolean partitioned;
+
+        /** */
+        private final boolean collocated;
+
+        /**
+         * @param partitioned Partitioned.
+         * @param collocated Collocated.
+         */
+        Type(boolean partitioned, boolean collocated) {
+            this.partitioned = partitioned;
+            this.collocated = collocated;
+        }
+
+        /**
+         * @return {@code true} If partitioned.
+         */
+        public boolean isPartitioned() {
+            return partitioned;
+        }
+
+        /**
+         * @return {@code true} If collocated.
+         */
+        public boolean isCollocated() {
+            return collocated;
+        }
+
+        /**
+         * @param partitioned Partitioned.
+         * @param collocated Collocated.
+         * @return Type.
+         */
+        static Type of(boolean partitioned, boolean collocated) {
+            if (collocated)
+                return partitioned ? Type.PARTITIONED_COLLOCATED : Type.REPLICATED;
+
+            assert partitioned;
+
+            return Type.PARTITIONED_NOT_COLLOCATED;
+        }
+    }
+
+    /**
+     * Affinity of a table relative to previous joined tables.
+     */
+    private enum Affinity {
+        /** */
+        NONE,
+
+        /** */
+        HAS_AFFINITY_CONDITION,
+
+        /** */
+        JOINED_WITH_COLLOCATED
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f5c63701/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
index c1666fd..b063528 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.internal.processors.query.h2.opt;
 
-import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicLong;
@@ -27,45 +25,25 @@ import org.apache.ignite.internal.util.lang.GridFilteredIterator;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.spi.indexing.IndexingQueryFilter;
-import org.h2.command.dml.Query;
-import org.h2.command.dml.Select;
-import org.h2.command.dml.SelectUnion;
 import org.h2.engine.Session;
-import org.h2.expression.Expression;
-import org.h2.expression.ExpressionColumn;
 import org.h2.index.BaseIndex;
-import org.h2.index.IndexCondition;
 import org.h2.index.ViewIndex;
 import org.h2.message.DbException;
 import org.h2.result.Row;
 import org.h2.result.SearchRow;
-import org.h2.table.Column;
-import org.h2.table.Table;
 import org.h2.table.TableFilter;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow.KEY_COL;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow.VAL_COL;
+import static org.apache.ignite.internal.processors.query.h2.opt.GridH2Collocation.buildCollocationModel;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.PREPARE;
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2TableFilterCollocation.PARTITIONED_COLLOCATED;
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2TableFilterCollocation.PARTITIONED_FIRST;
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2TableFilterCollocation.PARTITIONED_NOT_COLLOCATED;
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2TableFilterCollocation.REPLICATED;
 
 /**
  * Index base.
  */
 public abstract class GridH2IndexBase extends BaseIndex {
     /** */
-    private static final int MULTIPLIER_COLLOCATED = 1;
-
-    /** */
-    private static final int MULTIPLIER_UNICAST = 20;
-
-    /** */
-    private static final int MULTIPLIER_BROADCAST = 80;
-
-    /** */
     private static final AtomicLong idxIdGen = new AtomicLong();
 
     /** */
@@ -139,44 +117,6 @@ public abstract class GridH2IndexBase extends BaseIndex {
     }
 
     /**
-     * @param qry Query.
-     * @return {@code true} If it was proved that the query is fully collocated.
-     */
-    public static boolean isCollocated(Query qry) {
-        if (qry.isUnion()) {
-            SelectUnion union = (SelectUnion)qry;
-
-            return isCollocated(union.getLeft()) && isCollocated(union.getRight());
-        }
-
-        Select select = (Select)qry;
-
-        ArrayList<TableFilter> list = new ArrayList<>();
-
-        TableFilter f = select.getTopTableFilter();
-
-        assert f != null;
-
-        do {
-            list.add(f);
-
-            f = f.getJoin();
-        }
-        while (f != null);
-
-        TableFilter[] filters = list.toArray(new TableFilter[list.size()]);
-
-        Map<TableFilter,GridH2TableFilterCollocation> states = new HashMap<>();
-
-        for (int i = 0; i < filters.length; i++) {
-            if (getDistributedMultiplier0(filters[i].getMasks(), filters, i, states) != MULTIPLIER_COLLOCATED)
-                return false;
-        }
-
-        return true;
-    }
-
-    /**
      * @param ses Session.
      */
     private static void clearViewIndexCache(Session ses) {
@@ -188,310 +128,30 @@ public abstract class GridH2IndexBase extends BaseIndex {
 
     /**
      * @param ses Session.
-     * @param masks Masks.
      * @param filters All joined table filters.
      * @param filter Current filter.
      * @return Multiplier.
      */
-    public int getDistributedMultiplier(Session ses, int[] masks, TableFilter[] filters, int filter) {
+    public int getDistributedMultiplier(Session ses, TableFilter[] filters, int filter) {
         GridH2QueryContext qctx = GridH2QueryContext.get();
 
         // We do complex optimizations with respect to distributed joins only on prepare stage
         // because on run stage reordering of joined tables by Optimizer is explicitly disabled
         // and thus multiplier will be always the same, so it will not affect choice of index.
-        if (qctx == null || qctx.type() != PREPARE || !qctx.distributedJoins())
-            return MULTIPLIER_COLLOCATED;
+        // Query expressions can not be distributed as well.
+        if (qctx == null || qctx.type() != PREPARE || !qctx.distributedJoins() || ses.isPreparingQueryExpression())
+            return GridH2Collocation.MULTIPLIER_COLLOCATED;
 
-        // We have to clear this cache because normally sub-query plan cost does not depend on things
-        // other than index condition masks and sort order, but in our case it is.
+        // We have to clear this cache because normally sub-query plan cost does not depend on anything
+        // other than index condition masks and sort order, but in our case it can depend on order
+        // of previous table filters.
         clearViewIndexCache(ses);
 
-        Map<TableFilter,GridH2TableFilterCollocation> states = qctx.tableFilterStateCache();
+        assert filters != null;
 
-        // Need to do this clean up because subquery states can be outdated here.
-        clearPreviousSubQueryStates(filters, filter, states);
+        GridH2Collocation c = buildCollocationModel(qctx, ses.getSubQueryInfo(), filters, filter);
 
-        return getDistributedMultiplier0(masks, filters, filter, states);
-    }
-
-    /**
-     * @param masks Masks.
-     * @param filters All joined table filters.
-     * @param filter Current filter.
-     * @param states States map.
-     * @return Multiplier.
-     */
-    private static int getDistributedMultiplier0(int[] masks, TableFilter[] filters, int filter,
-        Map<TableFilter,GridH2TableFilterCollocation> states) {
-        assert states != null;
-
-        final TableFilter f = filters[filter];
-
-        if (!(f.getTable() instanceof GridH2Table)) {
-            GridH2TableFilterCollocation state = getStateForNonTable(f, states);
-
-            return state.isCollocated() ? MULTIPLIER_COLLOCATED : MULTIPLIER_BROADCAST;
-        }
-
-        GridH2Table tbl = (GridH2Table)f.getTable();
-
-        // Only partitioned tables will do distributed joins.
-        if (!tbl.isPartitioned()) {
-            states.put(f, REPLICATED);
-
-            return MULTIPLIER_COLLOCATED;
-        }
-
-        // If we are the first partitioned table in a join, then we are "base" for all the rest partitioned tables
-        // which will need to get remote result (if there is no affinity condition). Since this query is broadcasted
-        // to all the affinity nodes the "base" does not need to get remote results.
-        if (!findPartitionedTableBefore(filters, filter, states)) {
-            states.put(f, PARTITIONED_FIRST);
-
-            return MULTIPLIER_COLLOCATED;
-        }
-
-        // If we don't have affinity equality conditions then most probably we will have to broadcast.
-        if (!hasEqualityCondition(masks, affinityColumn(tbl))) {
-            states.put(f, PARTITIONED_NOT_COLLOCATED);
-
-            return MULTIPLIER_BROADCAST;
-        }
-
-        // If we have an affinity condition then we have to check if the whole join chain is collocated so far.
-        if (joinedWithCollocated(f, states)) {
-            states.put(f, PARTITIONED_COLLOCATED);
-
-            return MULTIPLIER_COLLOCATED;
-        }
-
-        // We are not collocated but at least we are going to unicast.
-        states.put(f, PARTITIONED_NOT_COLLOCATED);
-
-        return MULTIPLIER_UNICAST;
-    }
-
-    /**
-     * @param f Table filter.
-     * @param states States map.
-     * @return {@code true} If the given filter is joined with previous partitioned table filter which is
-     *      also collocated. Thus the whole join chain will be collocated.
-     */
-    private static boolean joinedWithCollocated(TableFilter f, Map<TableFilter,GridH2TableFilterCollocation> states) {
-        ArrayList<IndexCondition> idxConditions = f.getIndexConditions();
-
-        int affColId = affinityColumn((GridH2Table)f.getTable());
-
-        for (int i = 0; i < idxConditions.size(); i++) {
-            IndexCondition c = idxConditions.get(i);
-
-            if (c.getCompareType() == IndexCondition.EQUALITY &&
-                c.getColumn().getColumnId() == affColId && c.isEvaluatable()) {
-                Expression exp = c.getExpression();
-
-                exp = exp.getNonAliasExpression();
-
-                if (exp instanceof ExpressionColumn) {
-                    ExpressionColumn expCol = (ExpressionColumn)exp;
-
-                    // This is one of our previous joins.
-                    TableFilter prevJoin = expCol.getTableFilter();
-
-                    if (prevJoin != null) {
-                        GridH2TableFilterCollocation state = states.get(prevJoin);
-
-                        if (state == null)
-                            state = getStateForNonTable(prevJoin, states);
-
-                        if (state.isPartitioned() && state.isCollocated() && isAffinityColumn(prevJoin, expCol))
-                            return true;
-                    }
-                }
-            }
-        }
-
-        return false;
-    }
-
-    /**
-     * @param f Table filter.
-     * @param expCol Expression column.
-     * @return {@code true} It it is an affinity column.
-     */
-    private static boolean isAffinityColumn(TableFilter f, ExpressionColumn expCol) {
-        Column col = expCol.getColumn();
-
-        if (col == null)
-            return false;
-
-        Table t = col.getTable();
-
-        if (t.isView()) {
-            Query qry = ((ViewIndex)f.getIndex()).getQuery();
-
-            return isAffinityColumn(qry, expCol);
-        }
-
-        return t instanceof GridH2Table &&
-            col.getColumnId() == ((GridH2Table)t).getAffinityKeyColumn().column.getColumnId();
-    }
-
-    /**
-     * @param qry Query.
-     * @param expCol Expression column.
-     * @return {@code true} It it is an affinity column.
-     */
-    private static boolean isAffinityColumn(Query qry, ExpressionColumn expCol) {
-        if (qry.isUnion()) {
-            SelectUnion union = (SelectUnion)qry;
-
-            return isAffinityColumn(union.getLeft(), expCol) && isAffinityColumn(union.getRight(), expCol);
-        }
-
-        Expression exp = qry.getExpressions().get(expCol.getColumn().getColumnId()).getNonAliasExpression();
-
-        if (exp instanceof ExpressionColumn) {
-            expCol = (ExpressionColumn)exp;
-
-            return isAffinityColumn(expCol.getTableFilter(), expCol);
-        }
-
-        return false;
-    }
-
-    /**
-     * @param filters All joined table filters.
-     * @param filter Current filter.
-     * @param states States map.
-     */
-    private static void clearPreviousSubQueryStates(TableFilter[] filters, int filter,
-        Map<TableFilter,GridH2TableFilterCollocation> states) {
-        // We have to go back and clean up state for all the previous subqueries.
-        for (int i = filter - 1; i >= 0; i--) {
-            TableFilter f = filters[i];
-
-            if (f.getTable().isView())
-                states.put(f, null);
-            else
-                break;
-        }
-    }
-
-    /**
-     * @param filters All joined table filters.
-     * @param filter Current filter.
-     * @param states States map.
-     * @return {@code true} If there are partitioned table before.
-     */
-    private static boolean findPartitionedTableBefore(TableFilter[] filters, int filter,
-        Map<TableFilter,GridH2TableFilterCollocation> states) {
-        for (int i = 0; i < filter; i++) {
-            TableFilter prevFilter = filters[i];
-
-            GridH2TableFilterCollocation state = states.get(prevFilter);
-
-            if (state == null) // This can happen if previous filter is a subquery or function.
-                state = getStateForNonTable(prevFilter, states);
-
-            if (state.isPartitioned())
-                return true;
-        }
-
-        return false;
-    }
-
-    /**
-     * @param f Filter.
-     * @param states States map.
-     * @return Filter collocation.
-     */
-    private static GridH2TableFilterCollocation getStateForNonTable(TableFilter f,
-        Map<TableFilter,GridH2TableFilterCollocation> states) {
-        Table tbl = f.getTable();
-
-        GridH2TableFilterCollocation res;
-
-        if (tbl.isView()) {
-            Query qry = ((ViewIndex)f.getIndex()).getQuery();
-
-            res = getStateForSubQuery(qry, states);
-        }
-        else if (tbl instanceof GridH2Table)
-            throw new IllegalStateException("Table found: " + ((GridH2Table)tbl).identifier());
-        else {
-            // It is a some kind of function or system table.
-            res = REPLICATED;
-        }
-
-        assert res != null;
-
-        states.put(f, res);
-
-        return res;
-    }
-
-    /**
-     * @param qry Query.
-     * @param states States.
-     * @return Collocation.
-     */
-    private static GridH2TableFilterCollocation getStateForSubQuery(Query qry,
-        Map<TableFilter,GridH2TableFilterCollocation> states) {
-        if (!qry.isUnion())
-            return getStateForSubSelect((Select)qry, states);
-
-        SelectUnion union = (SelectUnion)qry;
-
-        GridH2TableFilterCollocation left = getStateForSubQuery(union.getLeft(), states);
-        GridH2TableFilterCollocation right = getStateForSubQuery(union.getRight(), states);
-
-        if (!left.isCollocated() || !right.isCollocated())
-            return PARTITIONED_NOT_COLLOCATED;
-
-        if (!left.isPartitioned() && !right.isPartitioned())
-            return REPLICATED;
-
-        if (left == PARTITIONED_FIRST && right == PARTITIONED_FIRST)
-            return PARTITIONED_FIRST;
-
-        return PARTITIONED_COLLOCATED;
-    }
-
-    /**
-     * @param select Select.
-     * @param states States.
-     * @return Collocation.
-     */
-    private static GridH2TableFilterCollocation getStateForSubSelect(Select select,
-        Map<TableFilter,GridH2TableFilterCollocation> states) {
-        int partitioned = -1;
-        int i = 0;
-
-        for (TableFilter f = select.getTopTableFilter(); f != null; f = f.getJoin()) {
-            GridH2TableFilterCollocation state = states.get(f);
-
-            if (state == null)
-                state = getStateForNonTable(f, states);
-
-            if (!state.isCollocated())
-                return PARTITIONED_NOT_COLLOCATED;
-
-            if (state.isPartitioned())
-                partitioned = i;
-
-            i++;
-        }
-
-        switch (partitioned) {
-            case -1:
-                return REPLICATED;
-
-            case 0:
-                return PARTITIONED_FIRST;
-
-            default:
-                return PARTITIONED_COLLOCATED;
-        }
+        return c.calculateMultiplier();
     }
 
     /**
@@ -502,15 +162,6 @@ public abstract class GridH2IndexBase extends BaseIndex {
         return tbl.getAffinityKeyColumn().column.getColumnId();
     }
 
-    /**
-     * @param masks Masks.
-     * @param colId Column ID.
-     * @return {@code true} If set of index conditions contains equality condition for the given column.
-     */
-    private static boolean hasEqualityCondition(int[] masks, int colId) {
-        return masks != null && (masks[colId] & IndexCondition.EQUALITY) == IndexCondition.EQUALITY;
-    }
-
     /** {@inheritDoc} */
     @Override public GridH2Table getTable() {
         return (GridH2Table)super.getTable();

http://git-wip-us.apache.org/repos/asf/ignite/blob/f5c63701/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
index b5243a7..29bbbaf 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
@@ -28,13 +28,11 @@ import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.spi.indexing.IndexingQueryFilter;
-import org.h2.table.TableFilter;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.LOCAL;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.MAP;
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.PREPARE;
 
 /**
  * Thread local SQL query context which is intended to be accessible from everywhere.
@@ -81,7 +79,7 @@ public class GridH2QueryContext {
     private int pageSize;
 
     /** */
-    private Map<TableFilter, GridH2TableFilterCollocation>  tableFilterStateCache;
+    private GridH2Collocation qryCollocation;
 
     /**
      * @param locNodeId Local node ID.
@@ -115,15 +113,17 @@ public class GridH2QueryContext {
     }
 
     /**
-     * @return Cache for table filter collocation states.
+     * @return Query collocation model.
      */
-    public Map<TableFilter, GridH2TableFilterCollocation> tableFilterStateCache() {
-        assert type() == PREPARE : type();
-
-        if (tableFilterStateCache == null)
-            tableFilterStateCache = new HashMap<>();
+    public GridH2Collocation queryCollocation() {
+        return qryCollocation;
+    }
 
-        return tableFilterStateCache;
+    /**
+     * @param qryCollocation Query collocation model.
+     */
+    public void queryCollocation(GridH2Collocation qryCollocation) {
+        this.qryCollocation = qryCollocation;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/f5c63701/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
index 774e110..17385e6 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
@@ -152,6 +152,13 @@ public class GridH2Table extends TableBase {
         return affKeyCol;
     }
 
+    /**
+     * @return Affinity key column ID.
+     */
+    public int getAffinityKeyColumnId() {
+        return affKeyCol.column.getColumnId();
+    }
+
     /** {@inheritDoc} */
     @Override public long getDiskSpaceUsed() {
         return 0;
@@ -907,7 +914,7 @@ public class GridH2Table extends TableBase {
         @Override public double getCost(Session ses, int[] masks, TableFilter[] filters, int filter,
             SortOrder sortOrder) {
             long rows = getRowCountApproximation();
-            int mul = delegate.getDistributedMultiplier(ses, masks, filters, filter);
+            int mul = delegate.getDistributedMultiplier(ses, filters, filter);
 
             return  mul * (rows + Constants.COST_ROW_OFFSET);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f5c63701/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableFilterCollocation.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableFilterCollocation.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableFilterCollocation.java
deleted file mode 100644
index 6b11352..0000000
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TableFilterCollocation.java
+++ /dev/null
@@ -1,64 +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.query.h2.opt;
-
-/**
- * Collocation of H2 table filter.
- */
-public enum GridH2TableFilterCollocation {
-    /** */
-    PARTITIONED_FIRST(true, true),
-
-    /** */
-    PARTITIONED_COLLOCATED(true, true),
-
-    /** */
-    PARTITIONED_NOT_COLLOCATED(true, false),
-
-    /** */
-    REPLICATED(false, true);
-
-    /** */
-    private final boolean partitioned;
-
-    /** */
-    private final boolean collocated;
-
-    /**
-     * @param partitioned Partitioned.
-     * @param collocated Collocated.
-     */
-    GridH2TableFilterCollocation(boolean partitioned, boolean collocated) {
-        this.partitioned = partitioned;
-        this.collocated = collocated;
-    }
-
-    /**
-     * @return {@code true} If partitioned.
-     */
-    public boolean isPartitioned() {
-        return partitioned;
-    }
-
-    /**
-     * @return {@code true} If collocated.
-     */
-    public boolean isCollocated() {
-        return collocated;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f5c63701/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index ed00e48..633cdf4 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -407,7 +407,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
     @Override public double getCost(Session ses, int[] masks, TableFilter[] filters, int filter, SortOrder sortOrder) {
         long rowCnt = getRowCountApproximation();
         double baseCost = getCostRangeIndex(masks, rowCnt, filters, filter, sortOrder);
-        int mul = getDistributedMultiplier(ses, masks, filters, filter);
+        int mul = getDistributedMultiplier(ses, filters, filter);
 
         return mul * baseCost;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f5c63701/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
index 9225967..034eebe 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
@@ -37,7 +37,7 @@ import org.h2.table.IndexColumn;
 import org.h2.util.IntArray;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase.isCollocated;
+import static org.apache.ignite.internal.processors.query.h2.opt.GridH2Collocation.isCollocated;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.AVG;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.CAST;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.COUNT;


[37/50] [abbrv] ignite git commit: ignite-split2 - test fix

Posted by sb...@apache.org.
ignite-split2 - test fix


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

Branch: refs/heads/ignite-1232
Commit: 2bdbd7e1080c7ea05146058d2b525057c78f9530
Parents: 2847880
Author: S.Vladykin <sv...@gridgain.com>
Authored: Sat Dec 26 08:22:46 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Sat Dec 26 08:22:46 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/query/IgniteSqlSplitterSelfTest.java       | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2bdbd7e1/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
index b996861..7a68392 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
@@ -241,7 +241,7 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
      * Test HAVING clause.
      */
     public void testHaving() {
-        IgniteCache<Integer, Integer> c = ignite(0).getOrCreateCache(cacheConfig("ints", true,
+        IgniteCache<Integer, Integer> c = ignite(0).getOrCreateCache(cacheConfig("having", true,
             Integer.class, Integer.class));
 
         try {


[42/50] [abbrv] ignite git commit: ignite-split2 - optimizations

Posted by sb...@apache.org.
ignite-split2 - optimizations


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

Branch: refs/heads/ignite-1232
Commit: bd5af493a84d373a81188af43f1fb3a6071bdfd7
Parents: e0c86cd
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Dec 29 16:22:08 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Dec 29 16:22:08 2015 +0300

----------------------------------------------------------------------
 .../query/h2/twostep/GridMergeIndex.java        | 24 +++++---
 .../h2/twostep/GridReduceQueryExecutor.java     | 63 +++++++++++---------
 2 files changed, 52 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bd5af493/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
index 2f06ac3..d3537da 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
@@ -18,16 +18,19 @@
 package org.apache.ignite.internal.processors.query.h2.twostep;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.ConcurrentModificationException;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
-import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Cursor;
 import org.h2.engine.Session;
 import org.h2.index.BaseIndex;
@@ -40,7 +43,6 @@ import org.h2.result.SortOrder;
 import org.h2.table.IndexColumn;
 import org.h2.table.TableFilter;
 import org.jetbrains.annotations.Nullable;
-import org.jsr166.ConcurrentHashMap8;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SQL_MERGE_TABLE_MAX_SIZE;
 import static org.apache.ignite.IgniteSystemProperties.getInteger;
@@ -56,7 +58,7 @@ public abstract class GridMergeIndex extends BaseIndex {
     private final AtomicInteger expRowsCnt = new AtomicInteger(0);
 
     /** Remaining rows per source node ID. */
-    private final ConcurrentMap<UUID, Counter> remainingRows = new ConcurrentHashMap8<>();
+    private Map<UUID, Counter> remainingRows;
 
     /** */
     private final AtomicBoolean lastSubmitted = new AtomicBoolean();
@@ -112,11 +114,19 @@ public abstract class GridMergeIndex extends BaseIndex {
     }
 
     /**
-     * @param nodeId Node ID.
+     * Set source nodes.
+     *
+     * @param nodes Nodes.
      */
-    public void addSource(UUID nodeId) {
-        if (remainingRows.put(nodeId, new Counter()) != null)
-            throw new IllegalStateException();
+    public void setSources(Collection<ClusterNode> nodes) {
+        assert remainingRows == null;
+
+        remainingRows = new HashMap<>(nodes.size(), 1f);
+
+        for (ClusterNode node : nodes) {
+            if (remainingRows.put(node.id(), new Counter()) != null)
+                throw new IllegalStateException("Duplicate node id: " + node.id());
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/bd5af493/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index a74681e..3319797 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -111,6 +111,9 @@ public class GridReduceQueryExecutor {
     private static final IgniteProductVersion DISTRIBUTED_JOIN_SINCE = IgniteProductVersion.fromString("1.4.0");
 
     /** */
+    private boolean oldNodesInTopology = true;
+
+    /** */
     private GridKernalContext ctx;
 
     /** */
@@ -400,7 +403,7 @@ public class GridReduceQueryExecutor {
      * @param topVer Topology version.
      * @param cctx Cache context for main space.
      * @param extraSpaces Extra spaces.
-     * @return Data nodes or {@code null} if repartitioning started and we need to retry..
+     * @return Data nodes or {@code null} if repartitioning started and we need to retry.
      */
     private Collection<ClusterNode> stableDataNodes(
         AffinityTopologyVersion topVer,
@@ -409,11 +412,13 @@ public class GridReduceQueryExecutor {
     ) {
         String space = cctx.name();
 
-        Set<ClusterNode> nodes = new HashSet<>(dataNodes(space, topVer));
+        Collection<ClusterNode> nodes = dataNodes(space, topVer);
 
         if (F.isEmpty(nodes))
             throw new CacheException("Failed to find data nodes for cache: " + space);
 
+        nodes = nodes.size() > 4 ? new HashSet<>(nodes) : new ArrayList<>(nodes);
+
         if (!F.isEmpty(extraSpaces)) {
             for (String extraSpace : extraSpaces) {
                 GridCacheContext<?,?> extraCctx = cacheContext(extraSpace);
@@ -469,21 +474,26 @@ public class GridReduceQueryExecutor {
      * @return {@code true} If there are old nodes in topology.
      */
     private boolean oldNodesInTopology() {
-        NavigableMap<IgniteProductVersion,Collection<ClusterNode>> m = ctx.discovery().topologyVersionMap();
-
-        if (F.isEmpty(m))
+        if (!oldNodesInTopology)
             return false;
 
-        for (Map.Entry<IgniteProductVersion,Collection<ClusterNode>> entry : m.entrySet()) {
-            if (entry.getKey().compareTo(DISTRIBUTED_JOIN_SINCE) >= 0)
-                return false;
+        NavigableMap<IgniteProductVersion,Collection<ClusterNode>> m = ctx.discovery().topologyVersionMap();
 
-            for (ClusterNode node : entry.getValue()) {
-                if (!node.isClient() && !node.isDaemon())
-                    return true;
+        if (!F.isEmpty(m)) {
+            for (Map.Entry<IgniteProductVersion,Collection<ClusterNode>> entry : m.entrySet()) {
+                if (entry.getKey().compareTo(DISTRIBUTED_JOIN_SINCE) >= 0)
+                    break;
+
+                for (ClusterNode node : entry.getValue()) {
+                    if (!node.isClient() && !node.isDaemon())
+                        return true;
+                }
             }
         }
 
+        // If we did not find old nodes, we assume that old node will not join further.
+        oldNodesInTopology = false;
+
         return false;
     }
 
@@ -516,7 +526,9 @@ public class GridReduceQueryExecutor {
 
             final String space = cctx.name();
 
-            final QueryRun r = new QueryRun(h2.connectionForSpace(space), qry.mapQueries().size(), qry.pageSize());
+            List<GridCacheSqlQuery> mapQrys = qry.mapQueries();
+
+            final QueryRun r = new QueryRun(h2.connectionForSpace(space), mapQrys.size(), qry.pageSize());
 
             AffinityTopologyVersion topVer = h2.readyTopologyVersion();
 
@@ -552,18 +564,16 @@ public class GridReduceQueryExecutor {
                 nodes = Collections.singleton(F.rand(nodes));
             }
 
-            int tblIdx = 0;
-
-            final boolean skipMergeTbl = !qry.explain() && qry.skipMergeTable();
+            final boolean skipMergeTbl = qry.skipMergeTable() && !qry.explain();
 
-            for (GridCacheSqlQuery mapQry : qry.mapQueries()) {
+            for (int i = 0; i < mapQrys.size(); i++) {
                 GridMergeIndex idx;
 
-                if (!skipMergeTbl || qry.explain()) {
+                if (!skipMergeTbl) {
                     GridMergeTable tbl;
 
                     try {
-                        tbl = createMergeTable(r.conn, mapQry, qry.explain());
+                        tbl = createMergeTable(r.conn, mapQrys.get(i), qry.explain());
                     }
                     catch (IgniteCheckedException e) {
                         throw new IgniteException(e);
@@ -571,18 +581,17 @@ public class GridReduceQueryExecutor {
 
                     idx = tbl.getScanIndex(null);
 
-                    fakeTable(r.conn, tblIdx++).innerTable(tbl);
+                    fakeTable(r.conn, i).innerTable(tbl);
                 }
                 else
                     idx = GridMergeIndexUnsorted.createDummy();
 
-                for (ClusterNode node : nodes)
-                    idx.addSource(node.id());
+                idx.setSources(nodes);
 
                 r.idxs.add(idx);
             }
 
-            r.latch = new CountDownLatch(r.idxs.size() * nodes.size());
+            r.latch = new CountDownLatch(mapQrys.size() * nodes.size());
 
             runs.put(qryReqId, r);
 
@@ -593,8 +602,6 @@ public class GridReduceQueryExecutor {
                             "Client node disconnected."));
                 }
 
-                List<GridCacheSqlQuery> mapQrys = qry.mapQueries();
-
                 if (qry.explain()) {
                     mapQrys = new ArrayList<>(qry.mapQueries().size());
 
@@ -652,13 +659,13 @@ public class GridReduceQueryExecutor {
                 Iterator<List<?>> resIter = null;
 
                 if (!retry) {
-                    if (skipMergeTbl && !qry.explain()) {
-                        List<List<?>> res = new ArrayList<>();
-
+                    if (skipMergeTbl) {
                         assert r.idxs.size() == 1 : r.idxs;
 
                         GridMergeIndex idx = r.idxs.get(0);
 
+                        List<List<?>> res = new ArrayList<>((int)idx.getRowCountApproximation());
+
                         Cursor cur = idx.findInStream(null, null);
 
                         while (cur.next()) {
@@ -747,7 +754,7 @@ public class GridReduceQueryExecutor {
                     U.warn(log, "Query run was already removed: " + qryReqId);
 
                 if (!skipMergeTbl) {
-                    for (int i = 0, mapQrys = qry.mapQueries().size(); i < mapQrys; i++)
+                    for (int i = 0; i < mapQrys.size(); i++)
                         fakeTable(null, i).innerTable(null); // Drop all merge tables.
                 }
             }


[40/50] [abbrv] ignite git commit: Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-split2

Posted by sb...@apache.org.
Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-split2


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

Branch: refs/heads/ignite-1232
Commit: e579c4c0dfce6b6e35171cd70119337c16dd35a4
Parents: 1115a79 f97dc9f
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Dec 28 22:11:01 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Dec 28 22:11:01 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    | 11 +++
 RELEASE_NOTES.txt                               |  2 +
 assembly/release-fabric-base.xml                |  6 ++
 assembly/release-sources.xml                    | 79 ++++++++++++++++++++
 modules/clients/pom.xml                         |  2 +-
 .../apache/ignite/internal/IgniteKernal.java    |  7 +-
 .../ignite/internal/IgniteNodeAttributes.java   |  3 +
 .../processors/rest/GridRestProcessor.java      |  4 +
 .../rest/client/message/GridClientNodeBean.java |  4 +-
 .../plugin/security/AuthenticationContext.java  | 33 ++++++--
 modules/platforms/cpp/DEVNOTES.txt              | 59 +++++++++++++++
 modules/platforms/cpp/README.txt                | 64 ++++------------
 modules/platforms/cpp/project/vs/ignite.sln     |  6 +-
 modules/platforms/cpp/project/vs/ignite.slnrel  | 10 +++
 .../platforms/cpp/project/vs/ignite_x86.slnrel  | 10 +++
 modules/platforms/dotnet/Apache.Ignite.sln      |  5 +-
 modules/platforms/dotnet/DEVNOTES.txt           | 18 +++++
 .../rest/protocols/http/jetty/rest.html         |  2 +-
 modules/scalar-2.10/pom.xml                     |  2 +-
 modules/scalar/pom.xml                          |  2 +-
 parent/pom.xml                                  |  4 +-
 pom.xml                                         | 23 +++---
 22 files changed, 276 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e579c4c0/parent/pom.xml
----------------------------------------------------------------------


[15/50] [abbrv] ignite git commit: ignite-split2 - messages

Posted by sb...@apache.org.
ignite-split2 - messages


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

Branch: refs/heads/ignite-1232
Commit: 9c110e8937a9a5b7931ed49d29080af6bf817c85
Parents: 94db68a
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Dec 8 03:16:24 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Dec 8 03:16:24 2015 +0300

----------------------------------------------------------------------
 .../ignite/codegen/MessageCodeGenerator.java    |  6 ++--
 .../cache/query/GridCacheSqlQuery.java          |  2 +-
 .../messages/GridQueryCancelRequest.java        |  2 +-
 .../twostep/messages/GridQueryFailResponse.java |  2 +-
 .../messages/GridQueryNextPageRequest.java      |  2 +-
 .../h2/twostep/msg/GridH2IndexRangeRequest.java | 24 ++++++++++++---
 .../twostep/msg/GridH2IndexRangeResponse.java   | 32 ++++++++++++++------
 7 files changed, 48 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9c110e89/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
----------------------------------------------------------------------
diff --git a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
index 7b98375..011eece 100644
--- a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
+++ b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
@@ -43,9 +43,6 @@ import org.apache.ignite.internal.GridDirectCollection;
 import org.apache.ignite.internal.GridDirectMap;
 import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.IgniteCodeGeneratingFail;
-import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateRequest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryEntry;
-import org.apache.ignite.internal.util.UUIDCollectionMessage;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
@@ -170,7 +167,7 @@ public class MessageCodeGenerator {
 
 //        gen.generateAll(true);
 
-        gen.generateAndWrite(GridNearAtomicUpdateRequest.class);
+//        gen.generateAndWrite(GridNearAtomicUpdateRequest.class);
 
 //        gen.generateAndWrite(DataStreamerEntry.class);
 
@@ -228,6 +225,7 @@ public class MessageCodeGenerator {
 //        gen.generateAndWrite(GridH2IndexRangeResponse.class);
 //        gen.generateAndWrite(GridH2RowRange.class);
 //        gen.generateAndWrite(GridH2RowRangeBounds.class);
+//        gen.generateAndWrite(GridH2QueryRequest.class);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/9c110e89/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
index 1127d7e..1870e66 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
@@ -269,4 +269,4 @@ public class GridCacheSqlQuery implements Message, GridCacheQueryMarshallable {
 
         return cp;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9c110e89/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryCancelRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryCancelRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryCancelRequest.java
index e7e1493..bd5d8e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryCancelRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryCancelRequest.java
@@ -112,4 +112,4 @@ public class GridQueryCancelRequest implements Message {
     @Override public byte fieldsCount() {
         return 1;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9c110e89/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java
index 5f47649..4503049 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryFailResponse.java
@@ -138,4 +138,4 @@ public class GridQueryFailResponse implements Message {
     @Override public byte fieldsCount() {
         return 2;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9c110e89/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageRequest.java
index 5ed6502..6597db5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageRequest.java
@@ -165,4 +165,4 @@ public class GridQueryNextPageRequest implements Message {
     @Override public byte fieldsCount() {
         return 3;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9c110e89/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java
index 91220c5..e72ff1e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeRequest.java
@@ -112,18 +112,24 @@ public class GridH2IndexRangeRequest implements Message {
 
         switch (writer.state()) {
             case 0:
-                if (!writer.writeCollection("bounds", bounds, MessageCollectionItemType.MSG))
+                if (!writer.writeInt("batchLookupId", batchLookupId))
                     return false;
 
                 writer.incrementState();
 
             case 1:
-                if (!writer.writeUuid("originNodeId", originNodeId))
+                if (!writer.writeCollection("bounds", bounds, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
             case 2:
+                if (!writer.writeUuid("originNodeId", originNodeId))
+                    return false;
+
+                writer.incrementState();
+
+            case 3:
                 if (!writer.writeLong("qryId", qryId))
                     return false;
 
@@ -143,7 +149,7 @@ public class GridH2IndexRangeRequest implements Message {
 
         switch (reader.state()) {
             case 0:
-                bounds = reader.readCollection("bounds", MessageCollectionItemType.MSG);
+                batchLookupId = reader.readInt("batchLookupId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -151,7 +157,7 @@ public class GridH2IndexRangeRequest implements Message {
                 reader.incrementState();
 
             case 1:
-                originNodeId = reader.readUuid("originNodeId");
+                bounds = reader.readCollection("bounds", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
                     return false;
@@ -159,6 +165,14 @@ public class GridH2IndexRangeRequest implements Message {
                 reader.incrementState();
 
             case 2:
+                originNodeId = reader.readUuid("originNodeId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 3:
                 qryId = reader.readLong("qryId");
 
                 if (!reader.isLastRead())
@@ -178,6 +192,6 @@ public class GridH2IndexRangeRequest implements Message {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 3;
+        return 4;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9c110e89/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java
index ee15636..cb8c1a4 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2IndexRangeResponse.java
@@ -155,30 +155,36 @@ public class GridH2IndexRangeResponse implements Message {
 
         switch (writer.state()) {
             case 0:
-                if (!writer.writeString("err", err))
+                if (!writer.writeInt("batchLookupId", batchLookupId))
                     return false;
 
                 writer.incrementState();
 
             case 1:
-                if (!writer.writeUuid("originNodeId", originNodeId))
+                if (!writer.writeString("err", err))
                     return false;
 
                 writer.incrementState();
 
             case 2:
-                if (!writer.writeLong("qryId", qryId))
+                if (!writer.writeUuid("originNodeId", originNodeId))
                     return false;
 
                 writer.incrementState();
 
             case 3:
-                if (!writer.writeCollection("ranges", ranges, MessageCollectionItemType.MSG))
+                if (!writer.writeLong("qryId", qryId))
                     return false;
 
                 writer.incrementState();
 
             case 4:
+                if (!writer.writeCollection("ranges", ranges, MessageCollectionItemType.MSG))
+                    return false;
+
+                writer.incrementState();
+
+            case 5:
                 if (!writer.writeByte("status", status))
                     return false;
 
@@ -198,7 +204,7 @@ public class GridH2IndexRangeResponse implements Message {
 
         switch (reader.state()) {
             case 0:
-                err = reader.readString("err");
+                batchLookupId = reader.readInt("batchLookupId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -206,7 +212,7 @@ public class GridH2IndexRangeResponse implements Message {
                 reader.incrementState();
 
             case 1:
-                originNodeId = reader.readUuid("originNodeId");
+                err = reader.readString("err");
 
                 if (!reader.isLastRead())
                     return false;
@@ -214,7 +220,7 @@ public class GridH2IndexRangeResponse implements Message {
                 reader.incrementState();
 
             case 2:
-                qryId = reader.readLong("qryId");
+                originNodeId = reader.readUuid("originNodeId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -222,7 +228,7 @@ public class GridH2IndexRangeResponse implements Message {
                 reader.incrementState();
 
             case 3:
-                ranges = reader.readCollection("ranges", MessageCollectionItemType.MSG);
+                qryId = reader.readLong("qryId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -230,6 +236,14 @@ public class GridH2IndexRangeResponse implements Message {
                 reader.incrementState();
 
             case 4:
+                ranges = reader.readCollection("ranges", MessageCollectionItemType.MSG);
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 5:
                 status = reader.readByte("status");
 
                 if (!reader.isLastRead())
@@ -249,6 +263,6 @@ public class GridH2IndexRangeResponse implements Message {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 5;
+        return 6;
     }
 }


[31/50] [abbrv] ignite git commit: Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-split2

Posted by sb...@apache.org.
Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-split2


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

Branch: refs/heads/ignite-1232
Commit: 3cf7587cd499e8bef8d3977dd78a1b797228b5e2
Parents: 9c1f61e d8c8214
Author: S.Vladykin <sv...@gridgain.com>
Authored: Thu Dec 17 00:26:36 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Thu Dec 17 00:26:36 2015 +0300

----------------------------------------------------------------------
 bin/ignitevisorcmd.bat                          |    1 +
 .../store/auto/CacheBinaryAutoStoreExample.java |    9 +-
 .../store/dummy/CacheDummyPersonStore.java      |  113 --
 .../store/dummy/CacheDummyStoreExample.java     |  133 --
 .../datagrid/store/dummy/package-info.java      |   22 -
 .../store/jdbc/CacheJdbcPersonStore.java        |   43 +-
 .../store/jdbc/CacheJdbcStoreExample.java       |   25 +-
 .../store/spring/CacheSpringPersonStore.java    |  131 --
 .../store/spring/CacheSpringStoreExample.java   |  147 --
 .../datagrid/store/spring/package-info.java     |   22 -
 .../examples/ScalarSnowflakeSchemaExample.scala |   31 +-
 .../ignite/examples/CacheExamplesSelfTest.java  |   10 +-
 modules/clients/pom.xml                         |    2 +-
 modules/core/pom.xml                            |    5 +
 .../src/main/java/org/apache/ignite/Ignite.java |   19 +
 .../java/org/apache/ignite/IgniteCache.java     |    8 +-
 .../org/apache/ignite/binary/BinaryObject.java  |   15 +
 .../org/apache/ignite/cache/CacheManager.java   |    2 +
 .../store/jdbc/CacheAbstractJdbcStore.java      |    2 +-
 .../configuration/CacheConfiguration.java       |   19 +-
 .../apache/ignite/internal/IgniteKernal.java    |    3 +-
 .../internal/binary/BinaryEnumObjectImpl.java   |    7 +
 .../internal/binary/BinaryObjectExImpl.java     |    7 +
 .../binary/BinaryObjectOffheapImpl.java         |    7 +
 .../processors/cache/GridCacheProcessor.java    |    4 +-
 .../CacheDataStructuresManager.java             |    5 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |    7 +-
 .../dht/GridPartitionedSingleGetFuture.java     |    7 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |   10 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   36 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |   25 +
 .../distributed/near/GridNearAtomicCache.java   |    3 +
 .../cache/query/GridCacheQueryRequest.java      |   12 +-
 .../cache/store/CacheOsStoreManager.java        |    2 +-
 .../transactions/IgniteTxLocalAdapter.java      |   23 +-
 .../service/GridServiceProcessor.java           |   30 +-
 .../internal/visor/query/VisorQueryJob.java     |    4 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |    2 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |    4 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   42 +-
 .../core/src/main/resources/ignite.properties   |    1 +
 .../CacheSerializableTransactionsTest.java      |   16 +
 .../cache/GridCacheAbstractFullApiSelfTest.java |    8 +
 ...IgniteCacheAtomicPutAllFailoverSelfTest.java |    1 +
 ...IgniteCacheBinaryEntryProcessorSelfTest.java |  255 +++
 .../GridCacheBinaryStoreAbstractSelfTest.java   |    2 +-
 ...ntNodeBinaryObjectMetadataMultinodeTest.java |    8 +-
 .../GridCacheQueueApiSelfAbstractTest.java      |   86 +-
 ...gniteAtomicLongChangingTopologySelfTest.java |    8 +-
 ...omicMultiNodeP2PDisabledFullApiSelfTest.java |    5 -
 ...ledFairAffinityMultiNodeFullApiSelfTest.java |    5 -
 .../near/NearCacheSyncUpdateTest.java           |  167 ++
 .../GridCacheRebalancingSyncSelfTest.java       |   12 +-
 .../BinaryTxCacheLocalEntriesSelfTest.java      |    2 +-
 .../GridServiceProcessorProxySelfTest.java      |    3 +-
 .../ServicePredicateAccessCacheTest.java        |  155 ++
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |   30 +-
 ...CommunicationRecoveryAckClosureSelfTest.java |   39 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |   16 +-
 .../testframework/junits/GridAbstractTest.java  |   10 +-
 .../testsuites/IgniteCacheTestSuite2.java       |    2 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    2 +
 .../IgniteSpiDiscoverySelfTestSuite.java        |    6 +
 .../common/include/ignite/common/concurrent.h   |    2 +-
 .../cpp/common/include/ignite/common/utils.h    |    1 -
 .../cpp/common/os/win/src/concurrent_os.cpp     |   26 +-
 modules/platforms/cpp/common/src/java.cpp       |   20 +-
 .../platforms/cpp/core-test/src/cache_test.cpp  |    4 +-
 modules/platforms/cpp/core/impl/doxygen.h       |   53 +
 .../core/include/ignite/binary/binary_type.h    |    4 +-
 .../cpp/core/include/ignite/cache/cache.h       |   12 +-
 .../include/ignite/cache/query/query_scan.h     |   28 +-
 .../include/ignite/cache/query/query_text.h     |   38 +-
 .../cpp/core/include/ignite/ignition.h          |   10 +-
 .../core/include/ignite/impl/cache/cache_impl.h |    6 +-
 .../cpp/core/include/ignite/impl/operations.h   |    8 +-
 modules/platforms/cpp/cpp.dxg                   |    6 +-
 .../native-client-test-cache-parallel-store.xml |    2 +-
 .../Config/native-client-test-cache-store.xml   |    6 +-
 .../Examples/ExamplesTest.cs                    |    4 +-
 .../Apache.Ignite.Core.csproj                   |   19 +
 .../Apache.Ignite.Core/Binary/Package-Info.cs   |   26 +
 .../Cache/Event/Package-Info.cs                 |   26 +
 .../Cache/Expiry/Package-Info.cs                |   26 +
 .../Apache.Ignite.Core/Cache/Package-Info.cs    |   26 +
 .../Cache/Query/Continuous/Package-Info.cs      |   26 +
 .../Cache/Query/Package-Info.cs                 |   26 +
 .../Cache/Store/Package-Info.cs                 |   26 +
 .../Apache.Ignite.Core/Cluster/Package-Info.cs  |   26 +
 .../Apache.Ignite.Core/Common/Package-Info.cs   |   26 +
 .../Apache.Ignite.Core/Compute/IComputeTask.cs  |   11 +-
 .../Apache.Ignite.Core/Compute/Package-Info.cs  |   26 +
 .../DataStructures/Package-Info.cs              |   26 +
 .../Datastream/Package-Info.cs                  |   26 +
 .../Apache.Ignite.Core/Events/Package-Info.cs   |   26 +
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |    8 -
 .../Lifecycle/Package-Info.cs                   |   26 +
 .../Messaging/Package-Info.cs                   |   26 +
 .../dotnet/Apache.Ignite.Core/Package-Info.cs   |   36 +
 .../Apache.Ignite.Core/Resource/Package-Info.cs |   26 +
 .../Apache.Ignite.Core/Services/Package-Info.cs |   26 +
 .../Transactions/Package-Info.cs                |   26 +
 modules/platforms/dotnet/Apache.Ignite.dxg      |    2 +-
 modules/platforms/dotnet/Apache.Ignite.sln      |    1 +
 modules/platforms/dotnet/dotnet.dxg             | 1795 ------------------
 .../Apache.Ignite.Examples.csproj               |    1 -
 .../Compute/ClosureExample.cs                   |    2 +-
 .../Compute/TaskExample.cs                      |    2 +-
 .../Datagrid/ContinuousQueryExample.cs          |    2 +-
 .../Datagrid/CrossPlatformExample.cs            |  205 --
 .../Datagrid/DataStreamerExample.cs             |    2 +-
 .../Datagrid/PutGetExample.cs                   |    2 +-
 .../Datagrid/QueryExample.cs                    |    2 +-
 .../Datagrid/StoreExample.cs                    |    2 +-
 .../Datagrid/TransactionExample.cs              |    2 +-
 .../Events/EventsExample.cs                     |    2 +-
 .../Services/ServicesExample.cs                 |    2 +-
 .../scala/org/apache/ignite/scalar/scalar.scala |   11 +-
 .../cache/IgnitePutGetBatchBenchmark.java       |   60 +
 .../cache/IgnitePutGetTxBatchBenchmark.java     |   86 +
 parent/pom.xml                                  |    1 +
 121 files changed, 1891 insertions(+), 2901 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3cf7587c/parent/pom.xml
----------------------------------------------------------------------


[34/50] [abbrv] ignite git commit: ignite-2205 - clear flag for query context

Posted by sb...@apache.org.
ignite-2205 - clear flag for query context


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

Branch: refs/heads/ignite-1232
Commit: d661395754eabe20a11f9745a502b7c3c74b0111
Parents: 9397c19
Author: S.Vladykin <sv...@gridgain.com>
Authored: Fri Dec 25 13:00:39 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Fri Dec 25 13:00:39 2015 +0300

----------------------------------------------------------------------
 .../query/h2/opt/GridH2QueryContext.java        | 25 +++++++++++++++-----
 .../query/h2/opt/GridH2TreeIndex.java           |  3 +++
 2 files changed, 22 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d6613957/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
index d39d7ab..d5fc41d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
@@ -47,6 +47,9 @@ public class GridH2QueryContext {
     /** */
     private final Key key;
 
+    /** */
+    private volatile boolean cleared;
+
     /** Index snapshots. */
     @GridToStringInclude
     private Map<Long, Object> snapshots;
@@ -357,7 +360,7 @@ public class GridH2QueryContext {
         qctx.remove();
 
         if (!onlyThreadLoc && x.key.type == MAP)
-            doClear(x.key);
+            doClear(x.key, false);
     }
 
     /**
@@ -367,13 +370,14 @@ public class GridH2QueryContext {
      * @param type Query type.
      */
     public static void clear(UUID locNodeId, UUID nodeId, long qryId, GridH2QueryType type) {
-        doClear(new Key(locNodeId, nodeId, qryId, type));
+        doClear(new Key(locNodeId, nodeId, qryId, type), false);
     }
 
     /**
      * @param key Context key.
+     * @param nodeStop Node is stopping.
      */
-    private static void doClear(Key key) {
+    private static void doClear(Key key, boolean nodeStop) {
         assert key.type == MAP : key.type;
 
         GridH2QueryContext x = qctxs.remove(key);
@@ -381,6 +385,8 @@ public class GridH2QueryContext {
         if (x == null)
             return;
 
+        x.cleared = true;
+
         assert x.key.equals(key);
 
         if (!F.isEmpty(x.snapshots)) {
@@ -392,20 +398,27 @@ public class GridH2QueryContext {
 
         List<GridReservable> r = x.reservations;
 
-        if (!F.isEmpty(r)) {
+        if (!nodeStop && !F.isEmpty(r)) {
             for (int i = 0; i < r.size(); i++)
                 r.get(i).release();
         }
     }
 
     /**
+     * @return {@code true} If the context is cleared.
+     */
+    public boolean isCleared() {
+        return cleared;
+    }
+
+    /**
      * @param locNodeId Local node ID.
      * @param nodeId Dead node ID.
      */
     public static void clearAfterDeadNode(UUID locNodeId, UUID nodeId) {
         for (Key key : qctxs.keySet()) {
             if (key.locNodeId.equals(locNodeId) && key.nodeId.equals(nodeId))
-                doClear(key);
+                doClear(key, false);
         }
     }
 
@@ -415,7 +428,7 @@ public class GridH2QueryContext {
     public static void clearLocalNodeStop(UUID locNodeId) {
         for (Key key : qctxs.keySet()) {
             if (key.locNodeId.equals(locNodeId))
-                doClear(key);
+                doClear(key, true);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d6613957/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index 13414bd..95efc8e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -1409,6 +1409,9 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
             assert remainingRanges > 0;
 
             for (int attempt = 0;; attempt++) {
+                if (qctx.isCleared())
+                    throw new GridH2RetryException("Query is cancelled.");
+
                 if (kernalContext().isStopping())
                     throw new GridH2RetryException("Stopping node.");
 


[43/50] [abbrv] ignite git commit: Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-split2

Posted by sb...@apache.org.
Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-split2


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

Branch: refs/heads/ignite-1232
Commit: b9b983a7d9afca79ba5382bdede756b3226bdeee
Parents: bd5af49 f1f8cda
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Dec 29 16:22:24 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Dec 29 16:22:24 2015 +0300

----------------------------------------------------------------------
 assembly/release-fabric-base.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------



[10/50] [abbrv] ignite git commit: ignite-split2 - distributedJoins

Posted by sb...@apache.org.
ignite-split2 - distributedJoins


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

Branch: refs/heads/ignite-1232
Commit: 7a1d6079abcf03361c35931ca8278969b2104df9
Parents: 71f394c
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Dec 7 07:58:49 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Dec 7 07:58:49 2015 +0300

----------------------------------------------------------------------
 .../ignite/cache/query/SqlFieldsQuery.java      | 24 ++++++
 .../org/apache/ignite/cache/query/SqlQuery.java | 25 ++++++
 .../cache/query/GridCacheTwoStepQuery.java      | 28 ++++---
 .../processors/query/h2/IgniteH2Indexing.java   | 88 ++++++++++++++++++--
 .../query/h2/sql/GridSqlQuerySplitter.java      |  6 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  8 +-
 .../query/IgniteSqlSplitterSelfTest.java        |  6 +-
 7 files changed, 161 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7a1d6079/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 65f8eba..48dab6b 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
@@ -59,6 +59,9 @@ public final class SqlFieldsQuery extends Query<List<?>> {
     /** */
     private boolean enforceJoinOrder;
 
+    /** */
+    private boolean distributedJoins;
+
     /**
      * Constructs SQL fields query.
      *
@@ -170,6 +173,27 @@ public final class SqlFieldsQuery extends Query<List<?>> {
         return this;
     }
 
+    /**
+     * Specify if distributed joins are enabled for this query.
+     *
+     * @param distributedJoins Distributed joins enabled.
+     * @return {@code this} For chaining.
+     */
+    public SqlFieldsQuery setDistributedJoins(boolean distributedJoins) {
+        this.distributedJoins = distributedJoins;
+
+        return this;
+    }
+
+    /**
+     * Check if distributed joins are enabled for this query.
+     *
+     * @return {@code true} If distributed joind enabled.
+     */
+    public boolean isDistributedJoins() {
+        return distributedJoins;
+    }
+
     /** {@inheritDoc} */
     @Override public SqlFieldsQuery setPageSize(int pageSize) {
         return (SqlFieldsQuery)super.setPageSize(pageSize);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7a1d6079/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java
index be3b390..e05ff13 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java
@@ -43,6 +43,9 @@ public final class SqlQuery<K, V> extends Query<Cache.Entry<K, V>> {
     @GridToStringInclude
     private Object[] args;
 
+    /** */
+    private boolean distributedJoins;
+
     /**
      * Constructs query for the given type name and SQL query.
      *
@@ -142,11 +145,33 @@ public final class SqlQuery<K, V> extends Query<Cache.Entry<K, V>> {
 
     /**
      * @param type Type.
+     * @return {@code this} For chaining.
      */
     public SqlQuery setType(Class<?> type) {
         return setType(GridQueryProcessor.typeName(type));
     }
 
+    /**
+     * Specify if distributed joins are enabled for this query.
+     *
+     * @param distributedJoins Distributed joins enabled.
+     * @return {@code this} For chaining.
+     */
+    public SqlQuery setDistributedJoins(boolean distributedJoins) {
+        this.distributedJoins = distributedJoins;
+
+        return this;
+    }
+
+    /**
+     * Check if distributed joins are enabled for this query.
+     *
+     * @return {@code true} If distributed joind enabled.
+     */
+    public boolean isDistributedJoins() {
+        return distributedJoins;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(SqlQuery.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7a1d6079/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
index 42cde28..7d7715a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
@@ -51,7 +51,7 @@ public class GridCacheTwoStepQuery {
     private Set<String> tbls;
 
     /** */
-    private boolean fullCollocation;
+    private boolean distributedJoins;
 
     /** */
     private boolean skipMergeTbl;
@@ -66,11 +66,24 @@ public class GridCacheTwoStepQuery {
     }
 
     /**
-     * @param fullCollocation If it is a collocated query and no distributed joins can occur.
+     * Specify if distributed joins are enabled for this query.
+     *
+     * @param distributedJoins Distributed joins enabled.
      */
-    public void fullCollocation(boolean fullCollocation) {
-        this.fullCollocation = fullCollocation;
+    public void distributedJoins(boolean distributedJoins) {
+        this.distributedJoins = distributedJoins;
     }
+
+    /**
+     * Check if distributed joins are enabled for this query.
+     *
+     * @return {@code true} If distributed joind enabled.
+     */
+    public boolean distributedJoins() {
+        return distributedJoins;
+    }
+
+
     /**
      * @return {@code True} if reduce query can skip merge table creation and get data directly from merge index.
      */
@@ -86,13 +99,6 @@ public class GridCacheTwoStepQuery {
     }
 
     /**
-     * @return {@code true} If it is a collocated query and no distributed joins can occur.
-     */
-    public boolean fullCollocation() {
-        return fullCollocation;
-    }
-
-    /**
      * @return If this is explain query.
      */
     public boolean explain() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7a1d6079/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 802a0fa..067af47 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -105,7 +105,6 @@ import org.apache.ignite.internal.util.lang.IgniteInClosure2X;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeGuard;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.T3;
 import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.SB;
@@ -298,7 +297,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     private final ConcurrentHashMap<Thread, StatementCache> stmtCache = new ConcurrentHashMap<>();
 
     /** */
-    private final GridBoundedConcurrentLinkedHashMap<T3<String, String, Boolean>, TwoStepCachedQuery> twoStepCache =
+    private final GridBoundedConcurrentLinkedHashMap<TwoStepCachedQueryKey, TwoStepCachedQuery> twoStepCache =
         new GridBoundedConcurrentLinkedHashMap<>(TWO_STEP_QRY_CACHE_SIZE);
 
     /**
@@ -918,7 +917,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @param filter Filter.
      */
     private void initLocalQueryContext(IndexingQueryFilter filter) {
-        GridH2QueryContext.set(new GridH2QueryContext(nodeId, nodeId, 0, LOCAL).filter(filter));
+        GridH2QueryContext.set(new GridH2QueryContext(nodeId, nodeId, 0, LOCAL).filter(filter).distributedJoins(false));
     }
 
     /** {@inheritDoc} */
@@ -983,6 +982,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         fqry.setArgs(qry.getArgs());
         fqry.setPageSize(qry.getPageSize());
+        fqry.setDistributedJoins(qry.isDistributedJoins());
 
         final QueryCursor<List<?>> res = queryTwoStep(cctx, fqry);
 
@@ -1051,11 +1051,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         Connection c = connectionForSpace(space);
         final boolean enforceJoinOrder = qry.isEnforceJoinOrder();
+        final boolean distributedJoins = qry.isDistributedJoins();
+        final boolean groupByCollocated = qry.isCollocated();
 
         GridCacheTwoStepQuery twoStepQry;
         List<GridQueryFieldMetadata> meta;
 
-        final T3<String, String, Boolean> cachedQryKey = new T3<>(space, sqlQry, qry.isCollocated());
+        final TwoStepCachedQueryKey cachedQryKey = new TwoStepCachedQueryKey(space, sqlQry, groupByCollocated,
+            distributedJoins, enforceJoinOrder);
         TwoStepCachedQuery cachedQry = twoStepCache.get(cachedQryKey);
 
         if (cachedQry != null) {
@@ -1067,7 +1070,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             Session ses = session(c);
 
             GridH2QueryContext.set(new GridH2QueryContext(locNodeId, locNodeId, 0, PREPARE)
-                .distributedJoins(isPartitioned(cctx)));
+                .distributedJoins(distributedJoins && isPartitioned(cctx)));
             enforceJoinOrder(enforceJoinOrder);
             ses.setJoinBatchEnabled(false);
 
@@ -1088,7 +1091,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             try {
                 bindParameters(stmt, F.asList(qry.getArgs()));
 
-                twoStepQry = GridSqlQuerySplitter.split((JdbcPreparedStatement)stmt, qry.getArgs(), qry.isCollocated());
+                twoStepQry = GridSqlQuerySplitter.split((JdbcPreparedStatement)stmt, qry.getArgs(), qry.isCollocated(),
+                    distributedJoins);
 
                 meta = meta(stmt.getMetaData());
             }
@@ -1758,6 +1762,78 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
+     * Key for cached two-step query.
+     */
+    private static final class TwoStepCachedQueryKey {
+        /** */
+        private final String space;
+
+        /** */
+        private final String sql;
+
+        /** */
+        private final boolean groupByCollocated;
+
+        /** */
+        private final boolean distributedJoins;
+
+        /** */
+        private final boolean enforceJoinOrder;
+
+        /**
+         * @param space Space.
+         * @param sql Sql.
+         * @param groupByCollocated Collocated GROUP BY.
+         * @param distributedJoins Distributed joins enabled.
+         * @param enforceJoinOrder Enforce join order of tables.
+         */
+        private TwoStepCachedQueryKey(String space, String sql, boolean groupByCollocated, boolean distributedJoins,
+            boolean enforceJoinOrder) {
+            this.space = space;
+            this.sql = sql;
+            this.groupByCollocated = groupByCollocated;
+            this.distributedJoins = distributedJoins;
+            this.enforceJoinOrder = enforceJoinOrder;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TwoStepCachedQueryKey that = (TwoStepCachedQueryKey)o;
+
+            if (groupByCollocated != that.groupByCollocated)
+                return false;
+
+            if (distributedJoins != that.distributedJoins)
+                return false;
+
+            if (enforceJoinOrder != that.enforceJoinOrder)
+                return false;
+
+            if (space != null ? !space.equals(that.space) : that.space != null)
+                return false;
+
+            return sql.equals(that.sql);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int result = space != null ? space.hashCode() : 0;
+            result = 31 * result + sql.hashCode();
+            result = 31 * result + (groupByCollocated ? 1 : 0);
+            result = 31 * result + (distributedJoins ? 1 : 0);
+            result = 31 * result + (enforceJoinOrder ? 1 : 0);
+
+            return result;
+        }
+    }
+
+    /**
      * Cached two-step query.
      */
     private static final class TwoStepCachedQuery {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7a1d6079/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
index 0d41e95..1bdb26a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
@@ -144,12 +144,14 @@ public class GridSqlQuerySplitter {
      * @param stmt Prepared statement.
      * @param params Parameters.
      * @param collocatedGroupBy Whether the query has collocated GROUP BY keys.
+     * @param distributedJoins If distributed joins enabled.
      * @return Two step query.
      */
     public static GridCacheTwoStepQuery split(
         JdbcPreparedStatement stmt,
         Object[] params,
-        boolean collocatedGroupBy
+        final boolean collocatedGroupBy,
+        final boolean distributedJoins
     ) {
         if (params == null)
             params = GridCacheSqlQuery.EMPTY_PARAMS;
@@ -178,7 +180,7 @@ public class GridSqlQuerySplitter {
         // We do not have to look at each map query separately here, because if
         // the whole initial query is collocated, then all the map sub-queries
         // will be collocated as well.
-        res.fullCollocation(isCollocated(query(prepared)));
+        res.distributedJoins(distributedJoins && !isCollocated(query(prepared)));
 
         return res;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7a1d6079/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 134631c..f80539e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -610,8 +610,8 @@ public class GridReduceQueryExecutor {
 
                 boolean retry = false;
 
-                final boolean oldStyle = true; oldNodesInTopology(); // TODO =
-                final boolean distributedJoins = !qry.fullCollocation();
+                final boolean oldStyle = oldNodesInTopology();
+                final boolean distributedJoins = qry.distributedJoins();
 
                 if (send(nodes,
                     oldStyle ?
@@ -682,8 +682,8 @@ public class GridReduceQueryExecutor {
                     else {
                         UUID locNodeId = ctx.localNodeId();
 
-                        GridH2QueryContext.set(
-                            new GridH2QueryContext(locNodeId, locNodeId, qryReqId, REDUCE).pageSize(r.pageSize));
+                        GridH2QueryContext.set(new GridH2QueryContext(locNodeId, locNodeId, qryReqId, REDUCE)
+                            .pageSize(r.pageSize).distributedJoins(false));
 
                         h2.enforceJoinOrder(enforceJoinOrder);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7a1d6079/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
index 596e157..57df338 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
@@ -31,6 +31,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.GridRandom;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.X;
 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;
@@ -235,8 +236,11 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
                 c.put(key++, p);
             }
 
+            X.println("Plan : " + c.query(new SqlFieldsQuery("explain select count(*) from Person p, Organization o " +
+                "where p.orgId = o._key").setDistributedJoins(true)).getAll());
+
             assertEquals(15000L, c.query(new SqlFieldsQuery("select count(*) from Person p, Organization o " +
-                "where p.orgId = o._key")).getAll().get(0).get(0));
+                "where p.orgId = o._key").setDistributedJoins(true)).getAll().get(0).get(0));
         }
         finally {
             c.destroy();


[18/50] [abbrv] ignite git commit: Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-split2

Posted by sb...@apache.org.
Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-split2


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

Branch: refs/heads/ignite-1232
Commit: 1113c6f9e07067633bf1be0ac3f1b61326e5aad7
Parents: ebfddeb c077522
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Dec 8 04:08:57 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Dec 8 04:08:57 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |   1 +
 examples/config/example-default.xml             |   7 +
 .../store/auto/CacheBinaryAutoStoreExample.java | 158 ++++++
 .../datagrid/store/auto/package-info.java       |  22 +
 .../store/auto/CacheAutoStoreExample.java       |  93 +++-
 .../auto/CacheAutoStoreLoadDataExample.java     |  85 ----
 .../datagrid/store/auto/CacheConfig.java        |  81 ---
 .../datagrid/store/auto/DbH2ServerStartup.java  |  79 ---
 .../ignite/examples/util/DbH2ServerStartup.java |  79 +++
 .../ignite/examples/util/package-info.java      |  22 +
 .../store/jdbc/CacheAbstractJdbcStore.java      |  12 +-
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |   9 +
 .../configuration/CacheConfiguration.java       |  34 +-
 .../ignite/internal/MarshallerContextImpl.java  |   7 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |  51 +-
 .../dht/preloader/GridDhtPartitionDemander.java |   2 +-
 .../GridDhtPartitionsExchangeFuture.java        |   9 +-
 .../cache/transactions/IgniteTxAdapter.java     |   5 +
 .../cache/transactions/IgniteTxEntry.java       |   8 +-
 .../handlers/query/QueryCommandHandler.java     |   6 +-
 .../optimized/OptimizedMarshallerUtils.java     |  16 +-
 ...eAtomicEntryProcessorDeploymentSelfTest.java |   6 +-
 .../IgniteCacheEntryProcessorCallTest.java      | 497 +++++++++++++++++++
 .../dht/GridCacheTxNodeFailureSelfTest.java     |   2 +
 .../dht/GridNearCacheTxNodeFailureSelfTest.java |   4 -
 ...idCachePartitionedHitsAndMissesSelfTest.java |   4 +-
 .../config/GridTestProperties.java              |   3 +
 .../testframework/junits/GridAbstractTest.java  |   7 +
 .../IgniteBinaryObjectsCacheTestSuite3.java     |   2 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 .../testsuites/IgniteCacheTestSuite2.java       |   2 +
 ...cheDuplicateEntityConfigurationSelfTest.java | 126 +++++
 .../IgniteCacheQuerySelfTestSuite.java          |   4 +
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |   3 +-
 .../ignite/schema/ui/SchemaImportApp.java       |  14 +
 .../yardstick/cache/IgnitePutTxBenchmark.java   |  26 +-
 ...IgniteTransactionalWriteInvokeBenchmark.java |  16 +-
 .../apache/ignite/yarn/utils/package-info.java  |  22 +
 parent/pom.xml                                  |   4 +
 39 files changed, 1218 insertions(+), 312 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1113c6f9/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index cc34ebe,fc88c75..5e191cc
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@@ -29,7 -29,9 +29,8 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexingDisabledSelfTest;
  import org.apache.ignite.internal.processors.cache.GridCacheQueryInternalKeysSelfTest;
  import org.apache.ignite.internal.processors.cache.GridCacheQuerySerializationSelfTest;
 -import org.apache.ignite.internal.processors.cache.GridCacheReduceQueryMultithreadedSelfTest;
  import org.apache.ignite.internal.processors.cache.IgniteCacheCollocatedQuerySelfTest;
+ import org.apache.ignite.internal.processors.cache.IgniteCacheDuplicateEntityConfigurationSelfTest;
  import org.apache.ignite.internal.processors.cache.IgniteCacheFieldsQueryNoDataSelfTest;
  import org.apache.ignite.internal.processors.cache.IgniteCacheLargeResultSelfTest;
  import org.apache.ignite.internal.processors.cache.IgniteCacheNoClassQuerySelfTest;
@@@ -106,9 -112,11 +107,12 @@@ public class IgniteCacheQuerySelfTestSu
          // Parsing
          suite.addTestSuite(GridQueryParsingTest.class);
  
+         // Config.
+         suite.addTestSuite(IgniteCacheDuplicateEntityConfigurationSelfTest.class);
+ 
          // Queries tests.
          suite.addTestSuite(IgniteSqlSplitterSelfTest.class);
 +        suite.addTestSuite(IgniteCachePartitionedQuerySelfTest.class);
          suite.addTestSuite(GridCacheQueryIndexDisabledSelfTest.class);
          suite.addTestSuite(IgniteCacheQueryLoadSelfTest.class);
          suite.addTestSuite(IgniteCacheLocalQuerySelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/1113c6f9/parent/pom.xml
----------------------------------------------------------------------


[16/50] [abbrv] ignite git commit: ignite-split2 - fixes + more asserts

Posted by sb...@apache.org.
ignite-split2 - fixes + more asserts


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

Branch: refs/heads/ignite-1232
Commit: 4a46085e6111eec6e1ed8285d57269bc96cd822d
Parents: 9c110e8
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Dec 8 03:52:04 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Dec 8 03:52:04 2015 +0300

----------------------------------------------------------------------
 .../processors/query/h2/opt/GridH2RowFactory.java |  2 +-
 .../processors/query/h2/opt/GridH2TreeIndex.java  | 18 ++++++++++++++++--
 2 files changed, 17 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4a46085e/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowFactory.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowFactory.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowFactory.java
index 3817308..148fab8 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowFactory.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2RowFactory.java
@@ -48,7 +48,7 @@ public class GridH2RowFactory extends RowFactory {
     public static GridH2Row create(Value... data) {
         switch (data.length) {
             case 0:
-                throw new IllegalStateException();
+                throw new IllegalStateException("Zero columns row.");
 
             case 1:
                 return new RowKey(data[0]);

http://git-wip-us.apache.org/repos/asf/ignite/blob/4a46085e/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index 04fd233..98b0b6a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -285,6 +285,8 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
                     // This is the first request containing all the search rows.
                     ConcurrentNavigableMap<GridSearchRowPointer,GridH2Row> snapshot0 = qctx.getSnapshot(idxId);
 
+                    assert !msg.bounds().isEmpty() : "empty bounds";
+
                     src = new RangeSource(msg.bounds(), snapshot0);
                 }
                 else {
@@ -298,12 +300,16 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
 
                 int maxRows = qctx.pageSize();
 
+                assert maxRows > 0 : maxRows;
+
                 while (maxRows > 0) {
                     GridH2RowRange range = src.next(maxRows);
 
                     if (range == null)
                         break;
 
+                    ranges.add(range);
+
                     maxRows -= range.rows().size();
                 }
 
@@ -820,9 +826,13 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
         if (row == null)
             return null;
 
-        List<GridH2ValueMessage> vals = new ArrayList<>(row.getColumnCount());
+        int cols = row.getColumnCount();
+
+        assert cols > 0 : cols;
 
-        for (int i = 0; i < vals.size(); i++) {
+        List<GridH2ValueMessage> vals = new ArrayList<>(cols);
+
+        for (int i = 0; i < cols; i++) {
             try {
                 vals.add(GridH2ValueMessageFactory.toMessage(row.getValue(i)));
             }
@@ -850,6 +860,8 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
 
         Value[] vals = new Value[getTable().getColumns().length];
 
+        assert vals.length > 0;
+
         for (int i = 0; i < indexColumns.length; i++) {
             try {
                 vals[indexColumns[i].column.getColumnId()] = msg.values().get(i).value(ctx);
@@ -905,6 +917,8 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
 
         List<GridH2ValueMessage> vals = msg.values();
 
+        assert !F.isEmpty(vals) : vals;
+
         Value[] vals0 = new Value[vals.size()];
 
         for (int i = 0; i < vals0.length; i++) {


[14/50] [abbrv] ignite git commit: ignite-split2 - minor

Posted by sb...@apache.org.
ignite-split2 - minor


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

Branch: refs/heads/ignite-1232
Commit: 94db68ad926b6f60e09b6c732ba68ce023fb3831
Parents: f356f4f
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Dec 8 02:59:33 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Dec 8 02:59:33 2015 +0300

----------------------------------------------------------------------
 .../processors/query/h2/IgniteH2Indexing.java   |  2 +-
 .../processors/query/h2/opt/GridH2Table.java    | 27 +++++++++-----
 .../query/h2/sql/GridSqlQuerySplitter.java      | 15 ++------
 .../query/h2/twostep/GridMapQueryExecutor.java  |  2 --
 .../h2/twostep/GridReduceQueryExecutor.java     | 38 ++++++++++++--------
 5 files changed, 45 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/94db68ad/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 067af47..89bf44c 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -1091,7 +1091,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             try {
                 bindParameters(stmt, F.asList(qry.getArgs()));
 
-                twoStepQry = GridSqlQuerySplitter.split((JdbcPreparedStatement)stmt, qry.getArgs(), qry.isCollocated(),
+                twoStepQry = GridSqlQuerySplitter.split((JdbcPreparedStatement)stmt, qry.getArgs(), groupByCollocated,
                     distributedJoins);
 
                 meta = meta(stmt.getMetaData());

http://git-wip-us.apache.org/repos/asf/ignite/blob/94db68ad/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
index 8ee00b9..ac21807 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
@@ -259,7 +259,7 @@ public class GridH2Table extends TableBase {
             ses.addLock(this);
         }
 
-        if (snapshotEnabled && snapshotInLock())
+        if (snapshotInLock())
             snapshotIndexes(null);
 
         return false;
@@ -269,7 +269,10 @@ public class GridH2Table extends TableBase {
      * @return {@code true} If we must snapshot and release index snapshots in {@link #lock(Session, boolean, boolean)}
      * and {@link #unlock(Session)} methods.
      */
-    private static boolean snapshotInLock() {
+    private boolean snapshotInLock() {
+        if (!snapshotEnabled)
+            return false;
+
         GridH2QueryContext qctx = GridH2QueryContext.get();
 
         return qctx == null || // Outside of Ignite query context.
@@ -279,9 +282,11 @@ public class GridH2Table extends TableBase {
 
     /**
      * @param qctx Query context.
-     * @return Snapshots.
      */
-    public Object[] snapshotIndexes(GridH2QueryContext qctx) {
+    public void snapshotIndexes(GridH2QueryContext qctx) {
+        if (!snapshotEnabled)
+            return;
+
         Object[] snapshots;
 
         Lock l;
@@ -294,7 +299,7 @@ public class GridH2Table extends TableBase {
                 snapshots = doSnapshotIndexes(snapshots, qctx);
 
                 if (snapshots != null)
-                    return snapshots; // Reused successfully.
+                    return; // Reused successfully.
             }
 
             l = lock(true, waitTime);
@@ -321,8 +326,6 @@ public class GridH2Table extends TableBase {
         finally {
             unlock(l);
         }
-
-        return snapshots;
     }
 
     /**
@@ -377,6 +380,8 @@ public class GridH2Table extends TableBase {
      */
     @SuppressWarnings("unchecked")
     private Object[] doSnapshotIndexes(Object[] snapshots, GridH2QueryContext qctx) {
+        assert snapshotEnabled;
+
         if (snapshots == null) // Nothing to reuse, create new snapshots.
             snapshots = new Object[idxs.size() - 1];
 
@@ -434,7 +439,7 @@ public class GridH2Table extends TableBase {
         if (ses != null && !sessions.remove(ses))
             return;
 
-        if (snapshotEnabled && snapshotInLock())
+        if (snapshotInLock())
             releaseSnapshots();
     }
 
@@ -442,7 +447,11 @@ public class GridH2Table extends TableBase {
      * Releases snapshots.
      */
     public void releaseSnapshots() {
-        for (int i = 1, len = idxs.size(); i < len; i++)  // Release snapshots on all except first which is scan.
+        if (!snapshotEnabled)
+            return;
+
+        // Release snapshots on all except first which is scan.
+        for (int i = 1, len = idxs.size(); i < len; i++)
             index(i).releaseSnapshot();
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/94db68ad/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
index 1bdb26a..a50c210 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
@@ -45,6 +45,7 @@ import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunction
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlPlaceholder.EMPTY;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser.prepared;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser.query;
+import static org.apache.ignite.internal.processors.query.h2.twostep.GridReduceQueryExecutor.toArray;
 
 /**
  * Splits a single SQL query into two step map-reduce query.
@@ -367,7 +368,7 @@ public class GridSqlQuerySplitter {
             findParams(mapQry, params, new ArrayList<>(params.length), paramIdxs).toArray());
 
         map.columns(collectColumns(mapExps));
-        map.parameterIndexes(toIntArray(paramIdxs));
+        map.parameterIndexes(toArray(paramIdxs));
 
         res.addMapQuery(map);
 
@@ -378,23 +379,13 @@ public class GridSqlQuerySplitter {
         GridCacheSqlQuery rdc = new GridCacheSqlQuery(rdcQry.getSQL(),
             findParams(rdcQry, params, new ArrayList<>(), paramIdxs).toArray());
 
-        rdc.parameterIndexes(toIntArray(paramIdxs));
+        rdc.parameterIndexes(toArray(paramIdxs));
         res.skipMergeTbl(rdcQry.simpleQuery());
 
         return rdc;
     }
 
     /**
-     * @param arr Integer array.
-     * @return Primitive int array.
-     */
-    private static int[] toIntArray(IntArray arr) {
-        int[] res = new int[arr.size()];
-        arr.toArray(res);
-        return res;
-    }
-
-    /**
      * @param cols Columns from SELECT clause.
      * @return Map of columns with types.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/94db68ad/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index 4fa204f..a88b3d9 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -19,8 +19,6 @@ package org.apache.ignite.internal.processors.query.h2.twostep;
 
 import java.lang.reflect.Field;
 import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
 import java.util.AbstractCollection;
 import java.util.ArrayList;
 import java.util.Collection;

http://git-wip-us.apache.org/repos/asf/ignite/blob/94db68ad/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index f80539e..6e9bf21 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -108,7 +108,7 @@ public class GridReduceQueryExecutor {
     public static final byte QUERY_POOL = GridIoPolicy.SYSTEM_POOL;
 
     /** */
-    private static final IgniteProductVersion DISTRIBUTED_JOIN_SINCE = IgniteProductVersion.fromString("1.6.0");
+    private static final IgniteProductVersion DISTRIBUTED_JOIN_SINCE = IgniteProductVersion.fromString("1.4.0");
 
     /** */
     private GridKernalContext ctx;
@@ -514,15 +514,9 @@ public class GridReduceQueryExecutor {
 
             final long qryReqId = reqIdGen.incrementAndGet();
 
-            final QueryRun r = new QueryRun();
+            final String space = cctx.name();
 
-            r.pageSize = qry.pageSize() <= 0 ? GridCacheTwoStepQuery.DFLT_PAGE_SIZE : qry.pageSize();
-
-            r.idxs = new ArrayList<>(qry.mapQueries().size());
-
-            String space = cctx.name();
-
-            r.conn = (JdbcConnection)h2.connectionForSpace(space);
+            final QueryRun r = new QueryRun(h2.connectionForSpace(space), qry.mapQueries().size(), qry.pageSize());
 
             AffinityTopologyVersion topVer = h2.readyTopologyVersion();
 
@@ -1126,6 +1120,9 @@ public class GridReduceQueryExecutor {
         Message msg,
         @Nullable IgniteBiClosure<ClusterNode, Message, Message> specialize
     ) {
+        if (log.isDebugEnabled())
+            log.debug("Sending: [msg=" + msg + ", nodes=" + nodes + ", specialize=" + specialize + "]");
+
         return h2.send(GridTopic.TOPIC_QUERY, nodes, msg, specialize, locNodeHandler);
     }
 
@@ -1170,7 +1167,7 @@ public class GridReduceQueryExecutor {
      * @param ints Ints.
      * @return Array.
      */
-    private static int[] toArray(IntArray ints) {
+    public static int[] toArray(IntArray ints) {
         int[] res = new int[ints.size()];
 
         ints.toArray(res);
@@ -1186,7 +1183,7 @@ public class GridReduceQueryExecutor {
         if (m == null)
             return null;
 
-        Map<UUID,int[]> res = new HashMap<>(m.size());
+        Map<UUID,int[]> res = new HashMap<>(m.size(), 1f);
 
         for (Map.Entry<ClusterNode,IntArray> entry : m.entrySet())
             res.put(entry.getKey().id(), toArray(entry.getValue()));
@@ -1267,25 +1264,36 @@ public class GridReduceQueryExecutor {
     }
 
     /**
-     *
+     * Query run.
      */
     private static class QueryRun {
         /** */
-        private List<GridMergeIndex> idxs;
+        private final List<GridMergeIndex> idxs;
 
         /** */
         private CountDownLatch latch;
 
         /** */
-        private JdbcConnection conn;
+        private final JdbcConnection conn;
 
         /** */
-        private int pageSize;
+        private final int pageSize;
 
         /** Can be either CacheException in case of error or AffinityTopologyVersion to retry if needed. */
         private final AtomicReference<Object> state = new AtomicReference<>();
 
         /**
+         * @param conn Connection.
+         * @param idxsCnt Number of indexes.
+         * @param pageSize Page size.
+         */
+        private QueryRun(Connection conn, int idxsCnt, int pageSize) {
+            this.conn = (JdbcConnection)conn;
+            this.idxs = new ArrayList<>(idxsCnt);
+            this.pageSize = pageSize > 0 ? pageSize : GridCacheTwoStepQuery.DFLT_PAGE_SIZE;
+        }
+
+        /**
          * @param o Fail state object.
          * @param nodeId Node ID.
          */


[47/50] [abbrv] ignite git commit: ignite-split2 - separate pool for index requests2

Posted by sb...@apache.org.
ignite-split2 - separate pool for index requests2


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

Branch: refs/heads/ignite-1232
Commit: 4d8deba87192e06ef998270359173b3a206103c3
Parents: c5dd4bc
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Jan 11 18:31:52 2016 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Jan 11 18:31:52 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/managers/communication/GridIoManager.java       | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4d8deba8/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index 32c96ca..a5900da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -647,6 +647,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
                 case AFFINITY_POOL:
                 case UTILITY_CACHE_POOL:
                 case MARSH_CACHE_POOL:
+                case IDX_POOL:
                 {
                     if (msg.isOrdered())
                         processOrderedMessage(nodeId, msg, plc, msgC);


[26/50] [abbrv] ignite git commit: ignite-split2 - better tests

Posted by sb...@apache.org.
ignite-split2 - better tests


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

Branch: refs/heads/ignite-1232
Commit: 084697b14492375e11d005e786b5758317be7860
Parents: 6a9a61a
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Dec 15 02:33:10 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Dec 15 02:33:10 2015 +0300

----------------------------------------------------------------------
 .../query/h2/opt/GridH2SpatialIndex.java        |  2 +-
 .../query/h2/opt/GridH2IndexBase.java           | 11 ++-
 .../query/h2/opt/GridH2TreeIndex.java           | 34 ++++----
 .../query/IgniteSqlSplitterSelfTest.java        | 81 ++++++++++++++------
 4 files changed, 79 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/084697b1/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SpatialIndex.java
----------------------------------------------------------------------
diff --git a/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SpatialIndex.java b/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SpatialIndex.java
index bc700a9..a2dc7b3 100644
--- a/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SpatialIndex.java
+++ b/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SpatialIndex.java
@@ -299,7 +299,7 @@ public class GridH2SpatialIndex extends GridH2IndexBase implements SpatialIndex
         }
         while (i.hasNext());
 
-        return filter(rows.iterator());
+        return filter(rows.iterator(), threadLocalFilter());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/084697b1/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
index 20795bc..81359c5 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
@@ -203,17 +203,16 @@ public abstract class GridH2IndexBase extends BaseIndex {
      * Filters rows from expired ones and using predicate.
      *
      * @param iter Iterator over rows.
+     * @param filter Optional filter.
      * @return Filtered iterator.
      */
-    protected Iterator<GridH2Row> filter(Iterator<GridH2Row> iter) {
+    protected Iterator<GridH2Row> filter(Iterator<GridH2Row> iter, IndexingQueryFilter filter) {
         IgniteBiPredicate<Object, Object> p = null;
 
-        IndexingQueryFilter f = filter();
-
-        if (f != null) {
+        if (filter != null) {
             String spaceName = getTable().spaceName();
 
-            p = f.forSpace(spaceName);
+            p = filter.forSpace(spaceName);
         }
 
         return new FilteringIterator(iter, U.currentTimeMillis(), p);
@@ -222,7 +221,7 @@ public abstract class GridH2IndexBase extends BaseIndex {
     /**
      * @return Filter for currently running query or {@code null} if none.
      */
-    protected IndexingQueryFilter filter() {
+    protected static IndexingQueryFilter threadLocalFilter() {
         GridH2QueryContext qctx = GridH2QueryContext.get();
 
         return qctx == null ? null : qctx.filter();

http://git-wip-us.apache.org/repos/asf/ignite/blob/084697b1/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index e21e6af..2677f07 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -68,7 +68,7 @@ import org.h2.index.Cursor;
 import org.h2.index.IndexCondition;
 import org.h2.index.IndexLookupBatch;
 import org.h2.index.IndexType;
-import org.h2.index.SingleRowCursor;
+import org.h2.message.DbException;
 import org.h2.result.Row;
 import org.h2.result.SearchRow;
 import org.h2.result.SortOrder;
@@ -287,7 +287,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
 
                     assert !msg.bounds().isEmpty() : "empty bounds";
 
-                    src = new RangeSource(msg.bounds(), snapshot0);
+                    src = new RangeSource(msg.bounds(), snapshot0, qctx.filter());
                 }
                 else {
                     // This is request to fetch next portion of data.
@@ -397,7 +397,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
 
     /** {@inheritDoc} */
     @Override public long getRowCount(@Nullable Session ses) {
-        IndexingQueryFilter f = filter();
+        IndexingQueryFilter f = threadLocalFilter();
 
         // Fast path if we don't need to perform any filtering.
         if (f == null || f.forSpace((getTable()).spaceName()) == null)
@@ -495,7 +495,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
     private Iterator<GridH2Row> doFind(@Nullable SearchRow first, boolean includeFirst, @Nullable SearchRow last) {
         ConcurrentNavigableMap<GridSearchRowPointer, GridH2Row> t = treeForRead();
 
-        return doFind0(t, first, includeFirst, last);
+        return doFind0(t, first, includeFirst, last, threadLocalFilter());
     }
 
     /**
@@ -503,13 +503,15 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
      * @param first Lower bound.
      * @param includeFirst Whether lower bound should be inclusive.
      * @param last Upper bound always inclusive.
+     * @param filter Filter.
      * @return Iterator over rows in given range.
      */
     private Iterator<GridH2Row> doFind0(
         ConcurrentNavigableMap<GridSearchRowPointer, GridH2Row> t,
         @Nullable SearchRow first,
         boolean includeFirst,
-        @Nullable SearchRow last
+        @Nullable SearchRow last,
+        IndexingQueryFilter filter
     ) {
         includeFirst &= first != null;
 
@@ -519,7 +521,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
         if (range == null)
             return new GridEmptyIterator<>();
 
-        return filter(range.values().iterator());
+        return filter(range.values().iterator(), filter);
     }
 
     /**
@@ -584,16 +586,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
 
     /** {@inheritDoc} */
     @Override public Cursor findFirstOrLast(Session ses, boolean first) {
-        ConcurrentNavigableMap<GridSearchRowPointer, GridH2Row> tree = treeForRead();
-
-        Iterator<GridH2Row> iter = filter(first ? tree.values().iterator() : tree.descendingMap().values().iterator());
-
-        GridSearchRowPointer res = null;
-
-        if (iter.hasNext())
-            res = iter.next();
-
-        return new SingleRowCursor((Row)res);
+        throw DbException.throwInternalError();
     }
 
     /** {@inheritDoc} */
@@ -1537,14 +1530,19 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
         /** */
         final ConcurrentNavigableMap<GridSearchRowPointer, GridH2Row> tree;
 
+        /** */
+        final IndexingQueryFilter filter;
+
         /**
          * @param bounds Bounds.
          * @param tree Snapshot.
          */
         RangeSource(
             Iterable<GridH2RowRangeBounds> bounds,
-            ConcurrentNavigableMap<GridSearchRowPointer, GridH2Row> tree
+            ConcurrentNavigableMap<GridSearchRowPointer, GridH2Row> tree,
+            IndexingQueryFilter filter
         ) {
+            this.filter = filter;
             this.tree = tree;
             boundsIter = bounds.iterator();
         }
@@ -1601,7 +1599,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
 
                 ConcurrentNavigableMap<GridSearchRowPointer,GridH2Row> t = tree != null ? tree : treeForRead();
 
-                curRange = doFind0(t, first, true, last);
+                curRange = doFind0(t, first, true, last, filter);
 
                 if (!curRange.hasNext()) {
                     // We have to return empty range.

http://git-wip-us.apache.org/repos/asf/ignite/blob/084697b1/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
index b7409c0..31eb421 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
@@ -25,6 +25,7 @@ import java.util.Random;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -210,43 +211,76 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void testNonCollocatedJoins() throws Exception {
-        IgniteCache<Integer, Object> c = ignite(0).getOrCreateCache(cacheConfig("persOrg", true,
-            Integer.class, Person.class, Integer.class, Organization.class));
+    public void testDistributedJoins() throws Exception {
+        CacheConfiguration ccfg = cacheConfig("persOrg", true,
+            Integer.class, Person.class, Integer.class, Organization.class);
+
+        IgniteCache<Integer, Object> c = ignite(0).getOrCreateCache(ccfg);
 
         try {
             awaitPartitionMapExchange();
 
-            int key = 0;
+            doTestDistributedJoins(c, 30, 100, 1000, false);
+            doTestDistributedJoins(c, 30, 100, 1000, true);
+        }
+        finally {
+            c.destroy();
+        }
+    }
 
-            for (int i = 0; i < 30; i++) {
-                Organization o = new Organization();
+    /**
+     * @param c Cache.
+     * @param orgs Number of organizations.
+     * @param persons Number of persons.
+     * @param pageSize Page size.
+     * @param enforceJoinOrder Enforce join order.
+     */
+    private void doTestDistributedJoins(IgniteCache<Integer, Object> c, int orgs, int persons, int pageSize,
+        boolean enforceJoinOrder) {
+        assertEquals(0, c.size(CachePeekMode.ALL));
 
-                o.name = "Org" + i;
+        int key = 0;
 
-                c.put(key++, o);
-            }
+        for (int i = 0; i < orgs; i++) {
+            Organization o = new Organization();
 
-            Random rnd = new GridRandom();
+            o.name = "Org" + i;
 
-            for (int i = 0; i < 100; i++) {
-                Person p = new Person();
+            c.put(key++, o);
+        }
 
-                p.name = "Person" + i;
-                p.orgId = rnd.nextInt(30);
+        Random rnd = new GridRandom();
 
-                c.put(key++, p);
-            }
+        for (int i = 0; i < persons; i++) {
+            Person p = new Person();
 
-            X.println("Plan : " + c.query(new SqlFieldsQuery("explain select count(*) from Person p, Organization o " +
-                "where p.orgId = o._key").setDistributedJoins(true)).getAll());
+            p.name = "Person" + i;
+            p.orgId = rnd.nextInt(orgs);
 
-            assertEquals(100L, c.query(new SqlFieldsQuery("select count(*) from Person p, Organization o " +
-                "where p.orgId = o._key").setDistributedJoins(true)).getAll().get(0).get(0));
-        }
-        finally {
-            c.destroy();
+            c.put(key++, p);
         }
+
+        String select = "select count(*) from Organization o, Person p where p.orgId = o._key";
+
+        String plan = (String)c.query(new SqlFieldsQuery("explain " + select)
+            .setDistributedJoins(true).setEnforceJoinOrder(enforceJoinOrder).setPageSize(pageSize))
+            .getAll().get(0).get(0);
+
+        X.println("Plan : " + plan);
+
+        if (enforceJoinOrder)
+            assertTrue(plan, plan.contains("batched:broadcast"));
+        else
+            assertTrue(plan, plan.contains("batched:unicast"));
+
+        assertEquals(Long.valueOf(persons), c.query(new SqlFieldsQuery(select).setDistributedJoins(true)
+            .setEnforceJoinOrder(enforceJoinOrder).setPageSize(pageSize)).getAll().get(0).get(0));
+
+        c.clear();
+
+        assertEquals(0, c.size(CachePeekMode.ALL));
+        assertEquals(0L, c.query(new SqlFieldsQuery(select).setDistributedJoins(true)
+            .setEnforceJoinOrder(enforceJoinOrder).setPageSize(pageSize)).getAll().get(0).get(0));
     }
 
     /**
@@ -277,7 +311,6 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
      *
      */
     public void testFunctionNpe() {
-        // TODO IGNITE-1886
         IgniteCache<Integer, User> userCache = ignite(0).createCache(
             cacheConfig("UserCache", true, Integer.class, User.class));
         IgniteCache<Integer, UserOrder> userOrderCache = ignite(0).createCache(


[20/50] [abbrv] ignite git commit: ignite-split2 - toString

Posted by sb...@apache.org.
ignite-split2 - toString


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

Branch: refs/heads/ignite-1232
Commit: 92480de43163dc483934b316001e27ed7e4d7477
Parents: 281fe7c
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Dec 8 04:59:46 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Dec 8 04:59:46 2015 +0300

----------------------------------------------------------------------
 .../processors/query/h2/twostep/msg/GridH2Array.java         | 5 +++++
 .../processors/query/h2/twostep/msg/GridH2Boolean.java       | 5 +++++
 .../internal/processors/query/h2/twostep/msg/GridH2Byte.java | 5 +++++
 .../processors/query/h2/twostep/msg/GridH2Bytes.java         | 7 +++++++
 .../processors/query/h2/twostep/msg/GridH2CacheObject.java   | 5 +++++
 .../internal/processors/query/h2/twostep/msg/GridH2Date.java | 5 +++++
 .../processors/query/h2/twostep/msg/GridH2Decimal.java       | 7 +++++++
 .../processors/query/h2/twostep/msg/GridH2Double.java        | 5 +++++
 .../processors/query/h2/twostep/msg/GridH2Float.java         | 5 +++++
 .../processors/query/h2/twostep/msg/GridH2Geometry.java      | 7 +++++++
 .../processors/query/h2/twostep/msg/GridH2Integer.java       | 5 +++++
 .../processors/query/h2/twostep/msg/GridH2JavaObject.java    | 7 +++++++
 .../internal/processors/query/h2/twostep/msg/GridH2Long.java | 5 +++++
 .../internal/processors/query/h2/twostep/msg/GridH2Null.java | 5 +++++
 .../processors/query/h2/twostep/msg/GridH2RowMessage.java    | 8 ++++++++
 .../query/h2/twostep/msg/GridH2RowRangeBounds.java           | 6 ++++++
 .../processors/query/h2/twostep/msg/GridH2Short.java         | 5 +++++
 .../processors/query/h2/twostep/msg/GridH2String.java        | 5 +++++
 .../internal/processors/query/h2/twostep/msg/GridH2Time.java | 5 +++++
 .../processors/query/h2/twostep/msg/GridH2Timestamp.java     | 5 +++++
 .../internal/processors/query/h2/twostep/msg/GridH2Uuid.java | 5 +++++
 21 files changed, 117 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Array.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Array.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Array.java
index 486050f..571f9ac 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Array.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Array.java
@@ -127,4 +127,9 @@ public class GridH2Array extends GridH2ValueMessage {
     @Override public byte fieldsCount() {
         return 1;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return String.valueOf(x);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Boolean.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Boolean.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Boolean.java
index 4162ba3..5681a66 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Boolean.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Boolean.java
@@ -110,4 +110,9 @@ public class GridH2Boolean extends GridH2ValueMessage {
     @Override public byte fieldsCount() {
         return 1;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return String.valueOf(x);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Byte.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Byte.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Byte.java
index f8e7e92..894794e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Byte.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Byte.java
@@ -111,4 +111,9 @@ public class GridH2Byte extends GridH2ValueMessage {
     @Override public byte fieldsCount() {
         return 1;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return String.valueOf(x);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Bytes.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Bytes.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Bytes.java
index 7fed4f0..29a52be 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Bytes.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Bytes.java
@@ -24,6 +24,8 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.h2.value.Value;
 import org.h2.value.ValueBytes;
 
+import static org.h2.util.StringUtils.convertBytesToHex;
+
 /**
  * H2 Bytes.
  */
@@ -111,4 +113,9 @@ public class GridH2Bytes extends GridH2ValueMessage {
     @Override public byte fieldsCount() {
         return 1;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "b_" + convertBytesToHex(b);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2CacheObject.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2CacheObject.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2CacheObject.java
index 38f6bbd..942ab7c 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2CacheObject.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2CacheObject.java
@@ -146,4 +146,9 @@ public class GridH2CacheObject extends GridH2ValueMessage {
     @Override public byte fieldsCount() {
         return 2;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return String.valueOf(obj);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Date.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Date.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Date.java
index 9c408eb..8025257 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Date.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Date.java
@@ -113,4 +113,9 @@ public class GridH2Date extends GridH2ValueMessage {
     @Override public byte fieldsCount() {
         return 1;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return String.valueOf(date);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Decimal.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Decimal.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Decimal.java
index 94284a6..a3ad444 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Decimal.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Decimal.java
@@ -26,6 +26,8 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.h2.value.Value;
 import org.h2.value.ValueDecimal;
 
+import static org.h2.util.StringUtils.convertBytesToHex;
+
 /**
  * H2 Decimal.
  */
@@ -133,4 +135,9 @@ public class GridH2Decimal extends GridH2ValueMessage {
     @Override public byte fieldsCount() {
         return 2;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return scale + "_" + convertBytesToHex(b);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Double.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Double.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Double.java
index de53d2b..2ceea8d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Double.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Double.java
@@ -111,4 +111,9 @@ public class GridH2Double extends GridH2ValueMessage {
     @Override public byte fieldsCount() {
         return 1;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return String.valueOf(x);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Float.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Float.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Float.java
index 67b99c4..6923470 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Float.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Float.java
@@ -111,4 +111,9 @@ public class GridH2Float extends GridH2ValueMessage {
     @Override public byte fieldsCount() {
         return 1;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return String.valueOf(x);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Geometry.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Geometry.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Geometry.java
index 960ea5e..0d118b4 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Geometry.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Geometry.java
@@ -25,6 +25,8 @@ import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.h2.value.Value;
 
+import static org.h2.util.StringUtils.convertBytesToHex;
+
 /**
  * H2 Geometry.
  */
@@ -132,4 +134,9 @@ public class GridH2Geometry extends GridH2ValueMessage {
     @Override public byte fieldsCount() {
         return 1;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "g_" + convertBytesToHex(b);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Integer.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Integer.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Integer.java
index dc4e484..16ebcad 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Integer.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Integer.java
@@ -134,4 +134,9 @@ public class GridH2Integer extends GridH2ValueMessage {
     @Override public int hashCode() {
         return x;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return String.valueOf(x);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2JavaObject.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2JavaObject.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2JavaObject.java
index 95211d7..b989171 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2JavaObject.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2JavaObject.java
@@ -24,6 +24,8 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.h2.value.Value;
 import org.h2.value.ValueJavaObject;
 
+import static org.h2.util.StringUtils.convertBytesToHex;
+
 /**
  * H2 Java Object.
  */
@@ -111,4 +113,9 @@ public class GridH2JavaObject extends GridH2ValueMessage {
     @Override public byte fieldsCount() {
         return 1;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "j_" + convertBytesToHex(b);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Long.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Long.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Long.java
index be6137b..3d360f0 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Long.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Long.java
@@ -111,4 +111,9 @@ public class GridH2Long extends GridH2ValueMessage {
     @Override public byte fieldsCount() {
         return 1;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return String.valueOf(x);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Null.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Null.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Null.java
index 83212fb..50a49ba 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Null.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Null.java
@@ -82,4 +82,9 @@ public class GridH2Null extends GridH2ValueMessage {
     @Override public byte fieldsCount() {
         return 0;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "NULL";
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowMessage.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowMessage.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowMessage.java
index 9b0adad..ca2c3bb 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowMessage.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowMessage.java
@@ -20,6 +20,8 @@ package org.apache.ignite.internal.processors.query.h2.twostep.msg;
 import java.nio.ByteBuffer;
 import java.util.List;
 import org.apache.ignite.internal.GridDirectCollection;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
@@ -31,6 +33,7 @@ import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 public class GridH2RowMessage implements Message {
     /** */
     @GridDirectCollection(Message.class)
+    @GridToStringInclude
     private List<GridH2ValueMessage> vals;
 
     /**
@@ -100,4 +103,9 @@ public class GridH2RowMessage implements Message {
     @Override public byte fieldsCount() {
         return 1;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridH2RowMessage.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowRangeBounds.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowRangeBounds.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowRangeBounds.java
index b3f83da..f5f70a3 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowRangeBounds.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2RowRangeBounds.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.query.h2.twostep.msg;
 
 import java.nio.ByteBuffer;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
@@ -174,4 +175,9 @@ public class GridH2RowRangeBounds implements Message {
     @Override public byte fieldsCount() {
         return 3;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridH2RowRangeBounds.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Short.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Short.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Short.java
index d0da1c3..ebeca9d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Short.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Short.java
@@ -111,4 +111,9 @@ public class GridH2Short extends GridH2ValueMessage {
     @Override public byte fieldsCount() {
         return 1;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return String.valueOf(x);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2String.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2String.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2String.java
index 270da11..f2f9fdc 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2String.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2String.java
@@ -113,4 +113,9 @@ public class GridH2String extends GridH2ValueMessage {
     @Override public byte fieldsCount() {
         return 1;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return x;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Time.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Time.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Time.java
index c8b9b3c..172d695 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Time.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Time.java
@@ -114,4 +114,9 @@ public class GridH2Time extends GridH2ValueMessage {
     @Override public byte fieldsCount() {
         return 1;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return String.valueOf(nanos);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Timestamp.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Timestamp.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Timestamp.java
index 6b2c081..b020799 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Timestamp.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Timestamp.java
@@ -131,4 +131,9 @@ public class GridH2Timestamp extends GridH2ValueMessage {
     @Override public byte fieldsCount() {
         return 2;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return date + "_" + nanos;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92480de4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Uuid.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Uuid.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Uuid.java
index 41f6532..fa9360b 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Uuid.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2Uuid.java
@@ -131,4 +131,9 @@ public class GridH2Uuid extends GridH2ValueMessage {
     @Override public byte fieldsCount() {
         return 2;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return ValueUuid.get(high, low).getString();
+    }
 }


[22/50] [abbrv] ignite git commit: Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-split2

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/dddb817b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
index 0000000,786b5b8..e3d763a
mode 000000,100644..100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
@@@ -1,0 -1,119 +1,116 @@@
+ /*
+  * 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.testsuites;
+ 
+ import junit.framework.TestSuite;
+ import org.apache.ignite.internal.processors.cache.CacheLocalQueryMetricsSelfTest;
+ import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsDistributedSelfTest;
+ import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsLocalSelfTest;
+ import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsDistributedSelfTest;
+ import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsLocalSelfTest;
+ import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexDisabledSelfTest;
+ import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexingDisabledSelfTest;
 -import org.apache.ignite.internal.processors.cache.GridCacheReduceQueryMultithreadedSelfTest;
+ import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectFieldsQuerySelfTest;
+ import org.apache.ignite.internal.processors.cache.IgniteCacheFieldsQueryNoDataSelfTest;
+ import org.apache.ignite.internal.processors.cache.IgniteCacheLargeResultSelfTest;
+ import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapTieredMultithreadedSelfTest;
+ import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingQueryErrorTest;
+ import org.apache.ignite.internal.processors.cache.IgniteCachePartitionedQueryMultiThreadedSelfTest;
+ import org.apache.ignite.internal.processors.cache.IgniteCacheQueryEvictsMultiThreadedSelfTest;
+ import org.apache.ignite.internal.processors.cache.IgniteCacheQueryMultiThreadedSelfTest;
+ import org.apache.ignite.internal.processors.cache.IgniteCacheQueryOffheapMultiThreadedSelfTest;
+ import org.apache.ignite.internal.processors.cache.binary.distributed.dht.GridCacheBinaryDuplicateIndexObjectPartitionedAtomicSelfTest;
+ import org.apache.ignite.internal.processors.cache.binary.distributed.dht.GridCacheBinaryDuplicateIndexObjectPartitionedTransactionalSelfTest;
+ import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicNearEnabledSelfTest;
+ import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicP2PDisabledSelfTest;
+ import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicSelfTest;
+ import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryLocalAtomicSelfTest;
+ import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionedOnlySelfTest;
+ import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedAtomicSelfTest;
+ import org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest;
+ import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest;
+ import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryTest;
+ import org.apache.ignite.internal.binary.BinaryMarshaller;
+ import org.apache.ignite.spi.communication.tcp.GridOrderedMessageCancelSelfTest;
+ import org.apache.ignite.testframework.config.GridTestProperties;
+ 
+ /**
+  * Cache query suite with binary marshaller.
+  */
+ public class IgniteBinaryCacheQueryTestSuite extends TestSuite {
+     /**
+      * @return Suite.
+      * @throws Exception In case of error.
+      */
+     public static TestSuite suite() throws Exception {
+         GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName());
+ 
+         TestSuite suite = new TestSuite("Grid Cache Query Test Suite using BinaryMarshaller");
+ 
+         // Parsing
+         suite.addTestSuite(GridQueryParsingTest.class);
+ 
+         // Queries tests.
+         suite.addTestSuite(GridCacheQueryIndexDisabledSelfTest.class);
+         suite.addTestSuite(IgniteCachePartitionedQueryMultiThreadedSelfTest.class);
+         suite.addTestSuite(IgniteCacheLargeResultSelfTest.class);
+         suite.addTestSuite(IgniteCacheQueryMultiThreadedSelfTest.class);
+         suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
+         suite.addTestSuite(IgniteCacheQueryOffheapMultiThreadedSelfTest.class);
+ 
+         suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);
 -        suite.addTestSuite(GridCacheReduceQueryMultithreadedSelfTest.class);
 -
+ 
+         // Fields queries.
+         suite.addTestSuite(IgniteCacheFieldsQueryNoDataSelfTest.class);
+         suite.addTestSuite(IgniteBinaryObjectFieldsQuerySelfTest.class);
+ 
+         // Continuous queries.
+         suite.addTestSuite(GridCacheContinuousQueryLocalAtomicSelfTest.class);
+         suite.addTestSuite(GridCacheContinuousQueryReplicatedAtomicSelfTest.class);
+         suite.addTestSuite(GridCacheContinuousQueryPartitionedOnlySelfTest.class);
+         suite.addTestSuite(GridCacheContinuousQueryAtomicSelfTest.class);
+         suite.addTestSuite(GridCacheContinuousQueryAtomicNearEnabledSelfTest.class);
+         suite.addTestSuite(GridCacheContinuousQueryAtomicP2PDisabledSelfTest.class);
+ 
+         suite.addTestSuite(GridCacheQueryIndexingDisabledSelfTest.class);
+ 
+         //Should be adjusted. Not ready to be used with BinaryMarshaller.
+         //suite.addTestSuite(GridCacheBinarySwapScanQuerySelfTest.class);
+ 
+         suite.addTestSuite(GridOrderedMessageCancelSelfTest.class);
+ 
+         // Ignite cache and H2 comparison.
+         suite.addTestSuite(BaseH2CompareQueryTest.class);
+         suite.addTestSuite(H2CompareBigQueryTest.class);
+ 
+         // Metrics tests
+         suite.addTestSuite(CacheLocalQueryMetricsSelfTest.class);
+         suite.addTestSuite(CachePartitionedQueryMetricsDistributedSelfTest.class);
+         suite.addTestSuite(CachePartitionedQueryMetricsLocalSelfTest.class);
+         suite.addTestSuite(CacheReplicatedQueryMetricsDistributedSelfTest.class);
+         suite.addTestSuite(CacheReplicatedQueryMetricsLocalSelfTest.class);
+ 
+         //Unmarshallig query test.
+         suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class);
+ 
+         suite.addTestSuite(GridCacheBinaryDuplicateIndexObjectPartitionedAtomicSelfTest.class);
+         suite.addTestSuite(GridCacheBinaryDuplicateIndexObjectPartitionedTransactionalSelfTest.class);
+ 
+         return suite;
+     }
+ }

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddb817b/modules/spring/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddb817b/parent/pom.xml
----------------------------------------------------------------------


[08/50] [abbrv] ignite git commit: ignite-split2 - fixes

Posted by sb...@apache.org.
ignite-split2 - fixes


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

Branch: refs/heads/ignite-1232
Commit: 750f146be9a032c3dbef0c4919592e57762b9f98
Parents: d86e0ae
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Dec 7 03:01:25 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Dec 7 03:01:25 2015 +0300

----------------------------------------------------------------------
 .../query/h2/opt/GridH2TreeIndex.java           | 158 ++++++++++++-------
 .../h2/twostep/GridReduceQueryExecutor.java     |  13 +-
 2 files changed, 106 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/750f146b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index 633cdf4..04fd233 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -54,12 +54,14 @@ import org.apache.ignite.internal.util.GridEmptyIterator;
 import org.apache.ignite.internal.util.offheap.unsafe.GridOffHeapSnapTreeMap;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeGuard;
 import org.apache.ignite.internal.util.snaptree.SnapTreeMap;
+import org.apache.ignite.internal.util.typedef.CIX2;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.logger.NullLogger;
+import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.spi.indexing.IndexingQueryFilter;
 import org.h2.engine.Session;
 import org.h2.index.Cursor;
@@ -78,8 +80,12 @@ import org.h2.value.ValueNull;
 import org.jetbrains.annotations.Nullable;
 
 import static java.util.Collections.emptyIterator;
+import static java.util.Collections.singletonList;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow.KEY_COL;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.MAP;
+import static org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2IndexRangeResponse.STATUS_ERROR;
+import static org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2IndexRangeResponse.STATUS_NOT_FOUND;
+import static org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2IndexRangeResponse.STATUS_OK;
 import static org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2RowRangeBounds.rangeBounds;
 import static org.h2.result.Row.MEMORY_CALCULATE;
 
@@ -106,6 +112,13 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
     /** */
     private final boolean snapshotEnabled;
 
+    /** */
+    private final CIX2<ClusterNode,Message> locNodeHandler = new CIX2<ClusterNode,Message>() {
+        @Override public void applyx(ClusterNode clusterNode, Message msg) throws IgniteCheckedException {
+            onMessage0(clusterNode.id(), msg);
+        }
+    };
+
     /**
      * Constructor with index initialization.
      *
@@ -195,23 +208,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
 
             msgLsnr = new GridMessageListener() {
                 @Override public void onMessage(UUID nodeId, Object msg) {
-                    ClusterNode node = kernalContext().discovery().node(nodeId);
-
-                    if (node == null)
-                        return;
-
-                    try {
-                        if (msg instanceof GridH2IndexRangeRequest)
-                            onIndexRangeRequest(node, (GridH2IndexRangeRequest)msg);
-                        else if (msg instanceof GridH2IndexRangeResponse)
-                            onIndexRangeResponse(node, (GridH2IndexRangeResponse)msg);
-                    }
-                    catch (Throwable th) {
-                        U.error(log, "Failed to handle message[nodeId=" + nodeId + ", msg=" + msg + "]", th);
-
-                        if (th instanceof Error)
-                            throw th;
-                    }
+                    onMessage0(nodeId, msg);
                 }
             };
 
@@ -225,6 +222,39 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
     }
 
     /**
+     * @param nodes Nodes.
+     * @param msg Message.
+     */
+    private void send(Collection<ClusterNode> nodes, Message msg) {
+        if (!getTable().rowDescriptor().indexing().send(msgTopic, nodes, msg, null, locNodeHandler))
+            throw new CacheException("Failed to send message to nodes: " + nodes + ".");
+    }
+
+    /**
+     * @param nodeId Source node ID.
+     * @param msg Message.
+     */
+    private void onMessage0(UUID nodeId, Object msg) {
+        ClusterNode node = kernalContext().discovery().node(nodeId);
+
+        if (node == null)
+            return;
+
+        try {
+            if (msg instanceof GridH2IndexRangeRequest)
+                onIndexRangeRequest(node, (GridH2IndexRangeRequest)msg);
+            else if (msg instanceof GridH2IndexRangeResponse)
+                onIndexRangeResponse(node, (GridH2IndexRangeResponse)msg);
+        }
+        catch (Throwable th) {
+            U.error(log, "Failed to handle message[nodeId=" + nodeId + ", msg=" + msg + "]", th);
+
+            if (th instanceof Error)
+                throw th;
+        }
+    }
+
+    /**
      * @return Kernal context.
      */
     private GridKernalContext kernalContext() {
@@ -239,60 +269,68 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
         GridH2QueryContext qctx = GridH2QueryContext.get(kernalContext().localNodeId(),
             msg.originNodeId(), msg.queryId(), MAP);
 
-        if (qctx == null) {
-            // TODO respond NOT_FOUND
+        GridH2IndexRangeResponse res = new GridH2IndexRangeResponse();
 
-            return;
-        }
+        res.originNodeId(msg.originNodeId());
+        res.queryId(msg.queryId());
+        res.batchLookupId(msg.batchLookupId());
 
-        RangeSource src;
+        if (qctx == null)
+            res.status(STATUS_NOT_FOUND);
+        else {
+            try {
+                RangeSource src;
 
-        if (msg.bounds() != null) {
-            // This is the first request containing all the search rows.
-            ConcurrentNavigableMap<GridSearchRowPointer,GridH2Row> snapshot0 = qctx.getSnapshot(idxId);
+                if (msg.bounds() != null) {
+                    // This is the first request containing all the search rows.
+                    ConcurrentNavigableMap<GridSearchRowPointer,GridH2Row> snapshot0 = qctx.getSnapshot(idxId);
 
-            src = new RangeSource(msg.bounds(), snapshot0);
-        }
-        else {
-            // This is request to fetch next portion of data.
-            src = qctx.getSource(node.id(), msg.batchLookupId());
+                    src = new RangeSource(msg.bounds(), snapshot0);
+                }
+                else {
+                    // This is request to fetch next portion of data.
+                    src = qctx.getSource(node.id(), msg.batchLookupId());
 
-            assert src != null;
-        }
+                    assert src != null;
+                }
 
-        List<GridH2RowRange> ranges = new ArrayList<>();
+                List<GridH2RowRange> ranges = new ArrayList<>();
 
-        int maxRows = qctx.pageSize();
+                int maxRows = qctx.pageSize();
 
-        while (maxRows > 0) {
-            GridH2RowRange range = src.next(maxRows);
+                while (maxRows > 0) {
+                    GridH2RowRange range = src.next(maxRows);
 
-            if (range == null)
-                break;
+                    if (range == null)
+                        break;
 
-            maxRows -= range.rows().size();
-        }
+                    maxRows -= range.rows().size();
+                }
 
-        if (src.hasMoreRows()) {
-            // Save source for future fetches.
-            if (msg.bounds() != null)
-                qctx.putSource(node.id(), msg.batchLookupId(), src);
-        }
-        else if (msg.bounds() == null) {
-            // Drop saved source.
-            qctx.putSource(node.id(), msg.batchLookupId(), null);
-        }
+                if (src.hasMoreRows()) {
+                    // Save source for future fetches.
+                    if (msg.bounds() != null)
+                        qctx.putSource(node.id(), msg.batchLookupId(), src);
+                }
+                else if (msg.bounds() == null) {
+                    // Drop saved source.
+                    qctx.putSource(node.id(), msg.batchLookupId(), null);
+                }
 
-        assert !ranges.isEmpty();
+                assert !ranges.isEmpty();
 
-        GridH2IndexRangeResponse res = new GridH2IndexRangeResponse();
+                res.ranges(ranges);
+                res.status(STATUS_OK);
+            }
+            catch (Throwable th) {
+                U.error(log, "Failed to process request: " + msg, th);
 
-        res.originNodeId(msg.originNodeId());
-        res.queryId(msg.queryId());
-        res.batchLookupId(msg.batchLookupId());
-        res.ranges(ranges);
+                res.error(th.getClass() + ": " + th.getMessage());
+                res.status(STATUS_ERROR);
+            }
+        }
 
-        // TODO send res
+        send(singletonList(node), res);
     }
 
     /**
@@ -1327,7 +1365,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
 
             assert remainingRanges > 0;
 
-            // TODO send req
+            send(singletonList(node), req);
         }
 
         /**
@@ -1367,14 +1405,14 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
                                 if (req.bounds() != null)
                                     req = createRequest(qctx, req.batchLookupId());
 
-                                // TODO request next by sending req
+                                send(singletonList(node), req);
                             }
                             else
                                 req = null;
 
                             return res;
 
-                        case GridH2IndexRangeResponse.STATUS_NOT_FOUND:
+                        case STATUS_NOT_FOUND:
                             if (req == null || req.bounds() == null) // We have already received the first response.
                                 throw new GridH2RetryException("Failure on remote node.");
 
@@ -1385,7 +1423,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
                                 throw new IgniteInterruptedException(e.getMessage());
                             }
 
-                            // TODO resend req
+                            send(singletonList(node), req);
 
                             break;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/750f146b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 3a8b8cb..134631c 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -159,6 +159,13 @@ public class GridReduceQueryExecutor {
     /** */
     private final GridSpinBusyLock busyLock;
 
+    /** */
+    private final CIX2<ClusterNode,Message> locNodeHandler = new CIX2<ClusterNode,Message>() {
+        @Override public void applyx(ClusterNode locNode, Message msg) {
+            h2.mapQueryExecutor().onMessage(locNode.id(), msg);
+        }
+    };
+
     /**
      * @param busyLock Busy lock.
      */
@@ -1119,11 +1126,7 @@ public class GridReduceQueryExecutor {
         Message msg,
         @Nullable IgniteBiClosure<ClusterNode, Message, Message> specialize
     ) {
-        return h2.send(GridTopic.TOPIC_QUERY, nodes, msg, specialize, new CIX2<ClusterNode,Message>() {
-            @Override public void applyx(ClusterNode locNode, Message msg) {
-                h2.mapQueryExecutor().onMessage(locNode.id(), msg);
-            }
-        });
+        return h2.send(GridTopic.TOPIC_QUERY, nodes, msg, specialize, locNodeHandler);
     }
 
     /**


[21/50] [abbrv] ignite git commit: ignite-split2 - msg

Posted by sb...@apache.org.
ignite-split2 - msg


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

Branch: refs/heads/ignite-1232
Commit: 16242b78dea8e99261c3aff15d571dcfb3c9fd0d
Parents: 92480de
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Dec 8 05:43:29 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Dec 8 05:43:29 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/codegen/MessageCodeGenerator.java   | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/16242b78/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
----------------------------------------------------------------------
diff --git a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
index 011eece..693d108 100644
--- a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
+++ b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
@@ -226,6 +226,7 @@ public class MessageCodeGenerator {
 //        gen.generateAndWrite(GridH2RowRange.class);
 //        gen.generateAndWrite(GridH2RowRangeBounds.class);
 //        gen.generateAndWrite(GridH2QueryRequest.class);
+//        gen.generateAndWrite(GridH2RowMessage.class);
     }
 
     /**


[09/50] [abbrv] ignite git commit: ignite-split2 - aff key fixed

Posted by sb...@apache.org.
ignite-split2 - aff key fixed


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

Branch: refs/heads/ignite-1232
Commit: 71f394c607d5befa78f9c79ca2ad9f98d3b4d7aa
Parents: 750f146
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Dec 7 03:37:08 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Dec 7 03:37:08 2015 +0300

----------------------------------------------------------------------
 .../cache/portable/CacheObjectBinaryProcessorImpl.java       | 8 ++++++++
 .../processors/cacheobject/IgniteCacheObjectProcessor.java   | 6 ++++++
 .../cacheobject/IgniteCacheObjectProcessorImpl.java          | 5 +++++
 .../ignite/internal/processors/query/GridQueryProcessor.java | 8 ++++++--
 4 files changed, 25 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/71f394c6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
index d172bca..23120da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectBinaryProcessorImpl.java
@@ -457,6 +457,14 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
     }
 
     /** {@inheritDoc} */
+    @Override public String affinityField(String keyType) {
+        if (portableCtx == null)
+            return null;
+
+        return portableCtx.affinityKeyFieldName(typeId(keyType));
+    }
+
+    /** {@inheritDoc} */
     @Override public BinaryObjectBuilder builder(String clsName) {
         return new BinaryObjectBuilderImpl(portableCtx, clsName);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/71f394c6/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
index 239b1b9..e552411 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
@@ -170,4 +170,10 @@ public interface IgniteCacheObjectProcessor extends GridProcessor {
      * @return Ignite binary interface.
      */
     public IgniteBinary binary();
+
+    /**
+     * @param keyType Key type name.
+     * @return Affinity filed name or {@code null}.
+     */
+    public String affinityField(String keyType);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71f394c6/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 0d36e0e..89f7ee8 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
@@ -86,6 +86,11 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
     }
 
     /** {@inheritDoc} */
+    @Override public String affinityField(String keyType) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
     @Override public IgniteBinary binary() {
         return noOpBinary;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/71f394c6/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 e5dd8c2..425c11d 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
@@ -235,8 +235,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         desc.keyClass(keyCls);
                     }
 
-                    desc.affinityKey(null); // TODO get affinity key field alias from config
-
                     TypeId typeId;
                     TypeId altTypeId = null;
 
@@ -247,6 +245,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                         if (valCls != null)
                             altTypeId = new TypeId(ccfg.getName(), valCls);
+
+                        // Need to setup affinity key for distributed joins.
+                        String affField = ctx.cacheObjects().affinityField(qryEntity.getKeyType());
+
+                        if (affField != null)
+                            desc.affinityKey(affField);
                     }
                     else {
                         processClassMeta(qryEntity, desc);


[28/50] [abbrv] ignite git commit: ignite-split2 - minor

Posted by sb...@apache.org.
ignite-split2 - minor


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

Branch: refs/heads/ignite-1232
Commit: 5cf4edc3ecf48ae3bc1dbf95193dc7eeed4ecdd0
Parents: 196346f
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Dec 15 05:03:55 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Dec 15 05:03:55 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/query/GridCacheTwoStepQuery.java    |  1 +
 .../processors/query/h2/opt/GridH2QueryContext.java      |  1 +
 .../processors/query/h2/opt/GridH2TreeIndex.java         | 11 +++++++----
 .../processors/query/IgniteSqlSplitterSelfTest.java      |  7 +++++--
 4 files changed, 14 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5cf4edc3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
index 7d7715a..dfc84b8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
@@ -183,6 +183,7 @@ public class GridCacheTwoStepQuery {
         cp.rdc = rdc.copy(args);
         cp.skipMergeTbl = skipMergeTbl;
         cp.pageSize = pageSize;
+        cp.distributedJoins = distributedJoins;
 
         for (int i = 0; i < mapQrys.size(); i++)
             cp.mapQrys.add(mapQrys.get(i).copy(args));

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cf4edc3/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
index c7d98c1..2c74f6d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
@@ -324,6 +324,7 @@ public class GridH2QueryContext {
      public static void set(GridH2QueryContext x) {
          assert qctx.get() == null;
 
+         // We need MAP query context to be available to other threads to run distributed joins.
          if (x.key.type == MAP && qctxs.putIfAbsent(x.key, x) != null)
              throw new IllegalStateException("Query context is already set.");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cf4edc3/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index eed21c4..7c7f715 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -1415,6 +1415,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
                                 if (req.bounds() != null)
                                     req = createRequest(qctx, req.batchLookupId());
 
+                                // Prefetch next page.
                                 send(singletonList(node), req);
                             }
                             else
@@ -1559,6 +1560,8 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
          * @return Range.
          */
         public GridH2RowRange next(int maxRows) {
+            assert maxRows > 0 : maxRows;
+
             for (;;) {
                 if (curRange.hasNext()) {
                     // Here we are getting last rows from previously partially fetched range.
@@ -1602,12 +1605,12 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
                 curRange = doFind0(t, first, true, last, filter);
 
                 if (!curRange.hasNext()) {
-                    // We have to return empty range.
-                    GridH2RowRange nextRange = new GridH2RowRange();
+                    // We have to return empty range here.
+                    GridH2RowRange emptyRange = new GridH2RowRange();
 
-                    nextRange.rangeId(curRangeId);
+                    emptyRange.rangeId(curRangeId);
 
-                    return nextRange;
+                    return emptyRange;
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cf4edc3/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
index 506be06..942a95c 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
@@ -223,8 +223,11 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
             doTestDistributedJoins(c, 30, 100, 1000, false);
             doTestDistributedJoins(c, 30, 100, 1000, true);
 
-            doTestDistributedJoins(c, 100, 2000, 10, false);
-            doTestDistributedJoins(c, 100, 2000, 10, true);
+            doTestDistributedJoins(c, 3, 10, 3, false);
+            doTestDistributedJoins(c, 3, 10, 3, true);
+
+            doTestDistributedJoins(c, 300, 2000, 5, false);
+            doTestDistributedJoins(c, 300, 2000, 5, true);
         }
         finally {
             c.destroy();


[30/50] [abbrv] ignite git commit: ignite-split2 - fixes for restart tests

Posted by sb...@apache.org.
ignite-split2 - fixes for restart tests


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

Branch: refs/heads/ignite-1232
Commit: 9c1f61e1aebab2dfa736818f16d8a1f152a23571
Parents: dad374f
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Dec 15 07:04:14 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Dec 15 07:04:14 2015 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java    |   2 +-
 .../query/h2/opt/GridH2TreeIndex.java           |  44 ++-
 .../query/h2/twostep/GridMapQueryExecutor.java  |  29 +-
 ...QueryNodeRestartDistributedJoinSelfTest.java | 396 +++++++++++++++++++
 4 files changed, 441 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9c1f61e1/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 9c1fff2..9d3cd31 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
@@ -556,7 +556,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             return;
 
         if (!busyLock.enterBusy())
-            throw new IllegalStateException("Failed to write to index (grid is stopping).");
+            return;
 
         try {
             if (coctx == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/9c1f61e1/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index 7c7f715..bffe4d0 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -227,7 +227,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
      */
     private void send(Collection<ClusterNode> nodes, Message msg) {
         if (!getTable().rowDescriptor().indexing().send(msgTopic, nodes, msg, null, locNodeHandler))
-            throw new CacheException("Failed to send message to nodes: " + nodes + ".");
+            throw new GridH2RetryException("Failed to send message to nodes: " + nodes + ".");
     }
 
     /**
@@ -310,7 +310,8 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
 
                     ranges.add(range);
 
-                    maxRows -= range.rows().size();
+                    if (range.rows() != null)
+                        maxRows -= range.rows().size();
                 }
 
                 if (src.hasMoreRows()) {
@@ -855,9 +856,14 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
 
         assert vals.length > 0;
 
+        List<GridH2ValueMessage> msgVals = msg.values();
+
         for (int i = 0; i < indexColumns.length; i++) {
+            if (i >= msgVals.size())
+                continue;
+
             try {
-                vals[indexColumns[i].column.getColumnId()] = msg.values().get(i).value(ctx);
+                vals[indexColumns[i].column.getColumnId()] = msgVals.get(i).value(ctx);
             }
             catch (IgniteCheckedException e) {
                 throw new CacheException(e);
@@ -1398,7 +1404,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
                     res = respQueue.poll(500, TimeUnit.MILLISECONDS);
                 }
                 catch (InterruptedException e) {
-                    throw new IgniteInterruptedException(e);
+                    throw new GridH2RetryException("Interrupted.");
                 }
 
                 if (res != null) {
@@ -1483,23 +1489,25 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
 
                 cursorRangeId = range.rangeId();
 
-                final Iterator<GridH2RowMessage> it = range.rows().iterator();
+                if (!F.isEmpty(range.rows())) {
+                    final Iterator<GridH2RowMessage> it = range.rows().iterator();
 
-                if (it.hasNext()) {
-                    cursor = new GridH2Cursor(new Iterator<Row>() {
-                        @Override public boolean hasNext() {
-                            return it.hasNext();
-                        }
+                    if (it.hasNext()) {
+                        cursor = new GridH2Cursor(new Iterator<Row>() {
+                            @Override public boolean hasNext() {
+                                return it.hasNext();
+                            }
 
-                        @Override public Row next() {
-                            // Lazily convert messages into real rows.
-                            return toRow(it.next());
-                        }
+                            @Override public Row next() {
+                                // Lazily convert messages into real rows.
+                                return toRow(it.next());
+                            }
 
-                        @Override public void remove() {
-                            throw new UnsupportedOperationException();
-                        }
-                    });
+                            @Override public void remove() {
+                                throw new UnsupportedOperationException();
+                            }
+                        });
+                    }
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9c1f61e1/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index ec54a27..878814a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -570,12 +570,22 @@ public class GridMapQueryExecutor {
                 qr.cancel();
             }
 
-            U.error(log, "Failed to execute local query.", e);
+            if (X.hasCause(e, GridH2RetryException.class)) {
+                try {
+                    sendRetry(node, reqId);
+                }
+                catch (IgniteCheckedException ex) {
+                    U.warn(log, "Failed to send retry message to node: " + node);
+                }
+            }
+            else {
+                U.error(log, "Failed to execute local query.", e);
 
-            sendError(node, reqId, e);
+                sendError(node, reqId, e);
 
-            if (e instanceof Error)
-                throw (Error)e;
+                if (e instanceof Error)
+                    throw (Error)e;
+            }
         }
         finally {
             // Release reserved partitions.
@@ -591,16 +601,13 @@ public class GridMapQueryExecutor {
      */
     private void sendError(ClusterNode node, long qryReqId, Throwable err) {
         try {
-            if (X.hasCause(err, GridH2RetryException.class)) {
-                sendRetry(node, qryReqId);
-
-                return;
-            }
-
             GridQueryFailResponse msg = new GridQueryFailResponse(qryReqId, err);
 
-            if (node.isLocal())
+            if (node.isLocal()) {
+                U.error(log, "Failed to run map query on local node.", err);
+
                 h2.reduceQueryExecutor().onMessage(ctx.localNodeId(), msg);
+            }
             else
                 ctx.io().send(node, GridTopic.TOPIC_QUERY, msg, QUERY_POOL);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9c1f61e1/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartDistributedJoinSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartDistributedJoinSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartDistributedJoinSelfTest.java
new file mode 100644
index 0000000..15f390d
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartDistributedJoinSelfTest.java
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.near;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicIntegerArray;
+import javax.cache.CacheException;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.GridRandom;
+import org.apache.ignite.internal.util.typedef.CAX;
+import org.apache.ignite.internal.util.typedef.F;
+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;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ * Test for distributed queries with node restarts.
+ */
+public class IgniteCacheQueryNodeRestartDistributedJoinSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final String QRY_0 = "select co.id, count(*) cnt\n" +
+        "from \"pe\".Person pe, \"pr\".Product pr, \"co\".Company co, \"pu\".Purchase pu\n" +
+        "where pe.id = pu.personId and pu.productId = pr.id and pr.companyId = co.id \n" +
+        "group by co.id order by cnt desc, co.id";
+
+    /** */
+    private static final String QRY_1 = "select pr.id, co.id\n" +
+        "from \"pr\".Product pr, \"co\".Company co\n" +
+        "where pr.companyId = co.id\n" +
+        "order by co.id, pr.id ";
+
+    /** */
+    private static final int GRID_CNT = 6;
+
+    /** */
+    private static final int PERS_CNT = 600;
+
+    /** */
+    private static final int PURCHASE_CNT = 6000;
+
+    /** */
+    private static final int COMPANY_CNT = 25;
+
+    /** */
+    private static final int PRODUCT_CNT = 100;
+
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        c.setDiscoverySpi(disco);
+
+        int i = 0;
+
+        CacheConfiguration<?, ?>[] ccs = new CacheConfiguration[4];
+
+        for (String name : F.asList("pe", "pu", "co", "pr")) {
+            CacheConfiguration<?, ?> cc = defaultCacheConfiguration();
+
+            cc.setName(name);
+            cc.setCacheMode(PARTITIONED);
+            cc.setBackups(2);
+            cc.setWriteSynchronizationMode(FULL_SYNC);
+            cc.setAtomicityMode(TRANSACTIONAL);
+            cc.setRebalanceMode(SYNC);
+            cc.setLongQueryWarningTimeout(15_000);
+            cc.setAffinity(new RendezvousAffinityFunction(false, 60));
+
+            switch (name) {
+                case "pe":
+                    cc.setIndexedTypes(
+                        Integer.class, Person.class
+                    );
+
+                    break;
+
+                case "pu":
+                    cc.setIndexedTypes(
+                        Integer.class, Purchase.class
+                    );
+
+                    break;
+
+                case "co":
+                    cc.setIndexedTypes(
+                        Integer.class, Company.class
+                    );
+
+                    break;
+
+                case "pr":
+                    cc.setIndexedTypes(
+                        Integer.class, Product.class
+                    );
+
+                    break;
+            }
+
+            ccs[i++] = cc;
+        }
+
+        c.setCacheConfiguration(ccs);
+
+        return c;
+    }
+
+    /**
+     *
+     */
+    private void fillCaches() {
+        IgniteCache<Integer, Company> co = grid(0).cache("co");
+
+        for (int i = 0; i < COMPANY_CNT; i++)
+            co.put(i, new Company(i));
+
+        IgniteCache<Integer, Product> pr = grid(0).cache("pr");
+
+        Random rnd = new GridRandom();
+
+        for (int i = 0; i < PRODUCT_CNT; i++)
+            pr.put(i, new Product(i, rnd.nextInt(COMPANY_CNT)));
+
+        IgniteCache<Integer, Person> pe = grid(0).cache("pe");
+
+        for (int i = 0; i < PERS_CNT; i++)
+            pe.put(i, new Person(i));
+
+        IgniteCache<Integer, Purchase> pu = grid(0).cache("pu");
+
+        for (int i = 0; i < PURCHASE_CNT; i++) {
+            int persId = rnd.nextInt(PERS_CNT);
+            int prodId = rnd.nextInt(PRODUCT_CNT);
+
+            pu.put(i, new Purchase(persId, prodId));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRestarts() throws Exception {
+        int duration = 90 * 1000;
+        int qryThreadNum = 4;
+        int restartThreadsNum = 2; // 4 + 2 = 6 nodes
+        final int nodeLifeTime = 2 * 1000;
+        final int logFreq = 10;
+
+        startGridsMultiThreaded(GRID_CNT);
+
+        final AtomicIntegerArray locks = new AtomicIntegerArray(GRID_CNT);
+
+        fillCaches();
+
+        final List<List<?>> pRes = grid(0).cache("pu").query(new SqlFieldsQuery(QRY_0)
+            .setDistributedJoins(true)).getAll();
+
+        Thread.sleep(3000);
+
+        assertEquals(pRes, grid(0).cache("pu").query(new SqlFieldsQuery(QRY_0)
+            .setDistributedJoins(true)).getAll());
+
+        final List<List<?>> rRes = grid(0).cache("co").query(new SqlFieldsQuery(QRY_1)
+            .setDistributedJoins(true)).getAll();
+
+        assertFalse(pRes.isEmpty());
+        assertFalse(rRes.isEmpty());
+
+        final AtomicInteger qryCnt = new AtomicInteger();
+
+        final AtomicBoolean qrysDone = new AtomicBoolean();
+
+        IgniteInternalFuture<?> fut1 = multithreadedAsync(new CAX() {
+            @Override public void applyx() throws IgniteCheckedException {
+                GridRandom rnd = new GridRandom();
+
+                while (!qrysDone.get()) {
+                    int g;
+
+                    do {
+                        g = rnd.nextInt(locks.length());
+                    }
+                    while (!locks.compareAndSet(g, 0, 1));
+
+                    if (rnd.nextBoolean()) { // Partitioned query.
+                        IgniteCache<?,?> cache = grid(g).cache("pu");
+
+                        SqlFieldsQuery qry = new SqlFieldsQuery(QRY_0).setDistributedJoins(true);
+
+                        boolean smallPageSize = rnd.nextBoolean();
+
+                        if (smallPageSize)
+                            qry.setPageSize(3);
+
+                        try {
+                            assertEquals(pRes, cache.query(qry).getAll());
+                        }
+                        catch (CacheException e) {
+                            assertTrue("On large page size must retry.", smallPageSize);
+
+                            boolean failedOnRemoteFetch = false;
+
+                            for (Throwable th = e; th != null; th = th.getCause()) {
+                                if (!(th instanceof CacheException))
+                                    continue;
+
+                                if (th.getMessage() != null &&
+                                    th.getMessage().startsWith("Failed to fetch data from node:")) {
+                                    failedOnRemoteFetch = true;
+
+                                    break;
+                                }
+                            }
+
+                            if (!failedOnRemoteFetch) {
+                                e.printStackTrace();
+
+                                fail("Must fail inside of GridResultPage.fetchNextPage or subclass.");
+                            }
+                        }
+                    }
+                    else {
+                        IgniteCache<?,?> cache = grid(g).cache("co");
+
+                        assertEquals(rRes, cache.query(new SqlFieldsQuery(QRY_1)
+                            .setDistributedJoins(true)).getAll());
+                    }
+
+                    locks.set(g, 0);
+
+                    int c = qryCnt.incrementAndGet();
+
+                    if (c % logFreq == 0)
+                        info("Executed queries: " + c);
+                }
+            }
+        }, qryThreadNum, "query-thread");
+
+        final AtomicInteger restartCnt = new AtomicInteger();
+
+        final AtomicBoolean restartsDone = new AtomicBoolean();
+
+        IgniteInternalFuture<?> fut2 = multithreadedAsync(new Callable<Object>() {
+            @SuppressWarnings({"BusyWait"})
+            @Override public Object call() throws Exception {
+                GridRandom rnd = new GridRandom();
+
+                while (!restartsDone.get()) {
+                    int g;
+
+                    do {
+                        g = rnd.nextInt(locks.length());
+                    }
+                    while (!locks.compareAndSet(g, 0, -1));
+
+                    log.info("Stop node: " + g);
+
+                    stopGrid(g);
+
+                    Thread.sleep(rnd.nextInt(nodeLifeTime));
+
+                    log.info("Start node: " + g);
+
+                    startGrid(g);
+
+                    Thread.sleep(rnd.nextInt(nodeLifeTime));
+
+                    locks.set(g, 0);
+
+                    int c = restartCnt.incrementAndGet();
+
+                    if (c % logFreq == 0)
+                        info("Node restarts: " + c);
+                }
+
+                return true;
+            }
+        }, restartThreadsNum, "restart-thread");
+
+        Thread.sleep(duration);
+
+        info("Stopping..");
+
+        restartsDone.set(true);
+
+        fut2.get();
+
+        info("Restarts stopped.");
+
+        qrysDone.set(true);
+
+        fut1.get();
+
+        info("Queries stopped.");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     *
+     */
+    private static class Person implements Serializable {
+        @QuerySqlField(index = true)
+        int id;
+
+        Person(int id) {
+            this.id = id;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class Purchase implements Serializable {
+        @QuerySqlField(index = true)
+        int personId;
+
+        @QuerySqlField(index = true)
+        int productId;
+
+        Purchase(int personId, int productId) {
+            this.personId = personId;
+            this.productId = productId;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class Company implements Serializable {
+        @QuerySqlField(index = true)
+        int id;
+
+        Company(int id) {
+            this.id = id;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class Product implements Serializable {
+        @QuerySqlField(index = true)
+        int id;
+
+        @QuerySqlField(index = true)
+        int companyId;
+
+        Product(int id, int companyId) {
+            this.id = id;
+            this.companyId = companyId;
+        }
+    }
+}
\ No newline at end of file


[04/50] [abbrv] ignite git commit: ignite-split2 - minor fixes

Posted by sb...@apache.org.
ignite-split2 - minor fixes


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

Branch: refs/heads/ignite-1232
Commit: 5a0bf09f8577bbe2ae20813053b18f030de170a6
Parents: 35e4b20
Author: S.Vladykin <sv...@gridgain.com>
Authored: Sat Dec 5 18:13:19 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Sat Dec 5 18:13:19 2015 +0300

----------------------------------------------------------------------
 .../query/h2/opt/GridH2Collocation.java         | 49 ++++++++++----------
 1 file changed, 25 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5a0bf09f/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java
index fd72b2b..13af99b 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java
@@ -35,7 +35,7 @@ import org.h2.table.TableFilter;
 /**
  * Collocation model for a query.
  */
-public class GridH2Collocation {
+public final class GridH2Collocation {
     /** */
     public static final int MULTIPLIER_COLLOCATED = 1;
 
@@ -113,7 +113,7 @@ public class GridH2Collocation {
      * @param childFilters New child filters.
      * @return {@code true} If child filters were updated.
      */
-    public boolean childFilters(TableFilter[] childFilters) {
+    private boolean childFilters(TableFilter[] childFilters) {
         assert childFilters != null;
         assert select == childFilters[0].getSelect();
 
@@ -303,7 +303,7 @@ public class GridH2Collocation {
                     TableFilter prevJoin = expCol.getTableFilter();
 
                     if (prevJoin != null) {
-                        GridH2Collocation co = children[indexOf(prevJoin)];
+                        GridH2Collocation co = child(indexOf(prevJoin));
 
                         assert co != null || isNotTableOrViewChild(-1, prevJoin);
 
@@ -395,8 +395,13 @@ public class GridH2Collocation {
         for (TableFilter f = select.getTopTableFilter(); f != null; f = f.getJoin()) {
             childFilters[i] = f;
 
+            GridH2Collocation c = child(i);
+
+            if (c == null)
+                child(i, c = new GridH2Collocation(this, i));
+
             if (f.getTable().isView())
-                children[i].finalizeChildFiltersOrder();
+                c.finalizeChildFiltersOrder();
 
             i++;
         }
@@ -419,8 +424,18 @@ public class GridH2Collocation {
             boolean needReset = false;
 
             for (int i = 0; i < childFilters.length; i++) {
-                if (childFilters[i].getTable().isView() && children[i].finalizeChildFiltersOrder())
-                    needReset = true;
+                Table t = childFilters[i].getTable();
+
+                if (t.isView() || t instanceof GridH2Table) {
+                    if (child(i) == null) {
+                        child(i, new GridH2Collocation(this, i));
+
+                        needReset = true;
+                    }
+
+                    if (t.isView() && child(i).finalizeChildFiltersOrder())
+                        needReset = true;
+                }
             }
 
             if (needReset)
@@ -494,8 +509,8 @@ public class GridH2Collocation {
      * @param idx Index.
      * @param child Child collocation.
      */
-    public void child(int idx, GridH2Collocation child) {
-        assert child(idx) == null;
+    private void child(int idx, GridH2Collocation child) {
+        assert children[idx] == null;
 
         children[idx] = child;
     }
@@ -504,25 +519,11 @@ public class GridH2Collocation {
      * @param idx Index.
      * @return Child collocation.
      */
-    public GridH2Collocation child(int idx) {
+    private GridH2Collocation child(int idx) {
         return children[idx];
     }
 
     /**
-     * @return Upper collocation.
-     */
-    public GridH2Collocation upper() {
-        return upper;
-    }
-
-    /**
-     * @return Filter.
-     */
-    public int filter() {
-        return filter;
-    }
-
-    /**
      * @param qctx Query context.
      * @param info Sub-query info.
      * @param filters Filters.
@@ -578,7 +579,7 @@ public class GridH2Collocation {
             }
 
             if (i == unions.size()) {
-                c = new GridH2Collocation(c.upper(), c.filter());
+                c = new GridH2Collocation(c.upper, c.filter);
 
                 unions.add(c);
 


[33/50] [abbrv] ignite git commit: ignite-split2 - fixes

Posted by sb...@apache.org.
ignite-split2 - fixes


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

Branch: refs/heads/ignite-1232
Commit: 9397c199122e2cdccab89b4026eb40e965962a90
Parents: b1a38a9
Author: S.Vladykin <sv...@gridgain.com>
Authored: Sun Dec 20 18:41:51 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Sun Dec 20 18:41:51 2015 +0300

----------------------------------------------------------------------
 .../processors/query/h2/IgniteH2Indexing.java   | 93 ++++++++------------
 .../query/h2/opt/GridH2CollocationModel.java    | 28 +++---
 .../query/h2/opt/GridH2TreeIndex.java           |  7 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  | 12 +--
 .../h2/twostep/GridReduceQueryExecutor.java     |  6 +-
 5 files changed, 68 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9397c199/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index b16b48e..23e7ec4 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -120,7 +120,6 @@ import org.apache.ignite.resources.LoggerResource;
 import org.apache.ignite.spi.indexing.IndexingQueryFilter;
 import org.h2.api.JavaObjectSerializer;
 import org.h2.command.CommandInterface;
-import org.h2.command.dml.OptimizerHints;
 import org.h2.engine.Session;
 import org.h2.engine.SysProperties;
 import org.h2.index.Index;
@@ -728,21 +727,12 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     @Override public GridQueryFieldsResult queryLocalSqlFields(@Nullable final String spaceName, final String qry,
         @Nullable final Collection<Object> params, final IndexingQueryFilter filters, boolean enforceJoinOrder)
         throws IgniteCheckedException {
-        initLocalQueryContext(filters);
+        Connection conn = connectionForSpace(spaceName);
 
-        try {
-            Connection conn = connectionForSpace(spaceName);
-
-            enforceJoinOrder(enforceJoinOrder);
-
-            ResultSet rs;
+        initLocalQueryContext(conn, enforceJoinOrder, filters);
 
-            try {
-                rs = executeSqlQueryWithTimer(spaceName, conn, qry, params, true);
-            }
-            finally {
-                enforceJoinOrder(false);
-            }
+        try {
+            ResultSet rs = executeSqlQueryWithTimer(spaceName, conn, qry, params, true);
 
             List<GridQueryFieldMetadata> meta = null;
 
@@ -888,25 +878,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * Executes query.
-     *
-     * @param space Space.
-     * @param qry Query.
-     * @param params Query parameters.
-     * @param tbl Target table of query to generate select.
-     * @return Result set.
-     * @throws IgniteCheckedException If failed.
-     */
-    private ResultSet executeQuery(String space, String qry, @Nullable Collection<Object> params,
-        TableDescriptor tbl) throws IgniteCheckedException {
-        Connection conn = connectionForThread(tbl.schema());
-
-        String sql = generateQuery(qry, tbl);
-
-        return executeSqlQueryWithTimer(space, conn, sql, params, true);
-    }
-
-    /**
      * Binds parameters to prepared statement.
      *
      * @param stmt Prepared statement.
@@ -923,12 +894,28 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
+     * @param conn Connection.
+     * @param enforceJoinOrder Enforce join order of tables.
      * @param filter Filter.
      */
-    private void initLocalQueryContext(IndexingQueryFilter filter) {
+    private void initLocalQueryContext(Connection conn, boolean enforceJoinOrder, IndexingQueryFilter filter) {
+        setupConnection(conn, false, enforceJoinOrder);
+
         GridH2QueryContext.set(new GridH2QueryContext(nodeId, nodeId, 0, LOCAL).filter(filter).distributedJoins(false));
     }
 
+    /**
+     * @param conn Connection to use.
+     * @param distributedJoins If distributed joins are enabled.
+     * @param enforceJoinOrder Enforce join order of tables.
+     */
+    public void setupConnection(Connection conn, boolean distributedJoins, boolean enforceJoinOrder) {
+        Session s = session(conn);
+
+        s.setForceJoinOrder(enforceJoinOrder);
+        s.setJoinBatchEnabled(distributedJoins);
+    }
+
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalSql(@Nullable String spaceName,
@@ -939,10 +926,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if (tbl == null)
             throw new CacheException("Failed to find SQL table for type: " + type.name());
 
-        initLocalQueryContext(filter);
+        String sql = generateQuery(qry, tbl);
+
+        Connection conn = connectionForThread(tbl.schema());
+
+        initLocalQueryContext(conn, false, filter);
 
         try {
-            ResultSet rs = executeQuery(spaceName, qry, params, tbl);
+            ResultSet rs = executeSqlQueryWithTimer(spaceName, conn, sql, params, true);
 
             return new KeyValIterator(rs);
         }
@@ -1026,18 +1017,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * @param enforce If {@code true}, then table join order will be enforced.
-     */
-    public void enforceJoinOrder(boolean enforce) {
-        OptimizerHints hints = null;
-        if (enforce) {
-            hints = new OptimizerHints();
-            hints.setJoinReorderEnabled(false);
-        }
-        OptimizerHints.set(hints);
-    }
-
-    /**
      * @param cctx Cache context.
      * @return {@code true} If the given cache is partitioned.
      */
@@ -1059,8 +1038,9 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         final String sqlQry = qry.getSql();
 
         Connection c = connectionForSpace(space);
+
         final boolean enforceJoinOrder = qry.isEnforceJoinOrder();
-        final boolean distributedJoins = qry.isDistributedJoins();
+        final boolean distributedJoins = qry.isDistributedJoins() && isPartitioned(cctx);
         final boolean groupByCollocated = qry.isCollocated();
 
         GridCacheTwoStepQuery twoStepQry;
@@ -1076,12 +1056,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
         else {
             final UUID locNodeId = ctx.localNodeId();
-            Session ses = session(c);
+
+            setupConnection(c, distributedJoins, enforceJoinOrder);
 
             GridH2QueryContext.set(new GridH2QueryContext(locNodeId, locNodeId, 0, PREPARE)
-                .distributedJoins(distributedJoins && isPartitioned(cctx)));
-            enforceJoinOrder(enforceJoinOrder);
-            ses.setJoinBatchEnabled(false);
+                .distributedJoins(distributedJoins));
 
             PreparedStatement stmt;
 
@@ -1092,8 +1071,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 throw new CacheException("Failed to parse query: " + sqlQry, e);
             }
             finally {
-                ses.setJoinBatchEnabled(true);
-                enforceJoinOrder(false);
                 GridH2QueryContext.clear(false);
             }
 
@@ -1429,7 +1406,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if (tbl == null)
             return -1;
 
-        ResultSet rs = executeSqlQuery(connectionForSpace(spaceName),
+        Connection conn = connectionForSpace(spaceName);
+
+        setupConnection(conn, false, false);
+
+        ResultSet rs = executeSqlQuery(conn,
             "SELECT COUNT(*) FROM " + tbl.fullTableName(), null, false);
 
         try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/9397c199/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2CollocationModel.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2CollocationModel.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2CollocationModel.java
index 2a1437c..95a2432 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2CollocationModel.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2CollocationModel.java
@@ -23,6 +23,7 @@ import java.util.List;
 import org.h2.command.dml.Query;
 import org.h2.command.dml.Select;
 import org.h2.command.dml.SelectUnion;
+import org.h2.expression.Comparison;
 import org.h2.expression.Expression;
 import org.h2.expression.ExpressionColumn;
 import org.h2.index.IndexCondition;
@@ -40,10 +41,10 @@ public final class GridH2CollocationModel {
     public static final int MULTIPLIER_COLLOCATED = 1;
 
     /** */
-    private static final int MULTIPLIER_UNICAST = 20;
+    private static final int MULTIPLIER_UNICAST = 50;
 
     /** */
-    private static final int MULTIPLIER_BROADCAST = 80;
+    private static final int MULTIPLIER_BROADCAST = 200;
 
     /** */
     private final GridH2CollocationModel upper;
@@ -285,7 +286,7 @@ public final class GridH2CollocationModel {
         for (int i = 0; i < idxConditions.size(); i++) {
             IndexCondition c = idxConditions.get(i);
 
-            if (c.getCompareType() == IndexCondition.EQUALITY &&
+            if (c.getCompareType() == Comparison.EQUAL &&
                 c.getColumn().getColumnId() == affColId && c.isEvaluatable()) {
                 affKeyConditionFound = true;
 
@@ -451,9 +452,13 @@ public final class GridH2CollocationModel {
         if (child == null && create && isChildTableOrView(i, null)) {
             TableFilter f = childFilters[i];
 
-            children[i] = child = f.getTable().isView() ?
-                buildCollocationModel(this, i, getSubQuery(f), null) :
-                createChildModel(this, i, null);
+            if (f.getTable().isView())
+                child = buildCollocationModel(this, i, getSubQuery(f), null);
+            else
+                child = createChildModel(this, i, null);
+
+            assert child != null;
+            assert children[i] == child;
         }
 
         return child;
@@ -558,10 +563,13 @@ public final class GridH2CollocationModel {
 
             SelectUnion union = (SelectUnion)qry;
 
-            GridH2CollocationModel a = buildCollocationModel(upper, filter, union.getLeft(), unions);
-            GridH2CollocationModel b = buildCollocationModel(upper, filter, union.getRight(), unions);
+            GridH2CollocationModel left = buildCollocationModel(upper, filter, union.getLeft(), unions);
+            GridH2CollocationModel right = buildCollocationModel(upper, filter, union.getRight(), unions);
+
+            assert left != null;
+            assert right != null;
 
-            return a == null ? b : a;
+            return upper != null ? upper : left;
         }
 
         Select select = (Select)qry;
@@ -586,7 +594,7 @@ public final class GridH2CollocationModel {
                 createChildModel(cm, i, null);
         }
 
-        return upper == null ? cm : null;
+        return upper != null ? upper : cm;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/9397c199/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index 5641222..13414bd 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -1423,7 +1423,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
 
                 if (res != null) {
                     switch (res.status()) {
-                        case GridH2IndexRangeResponse.STATUS_OK:
+                        case STATUS_OK:
                             List<GridH2RowRange> ranges0 = res.ranges();
 
                             remainingRanges -= ranges0.size();
@@ -1454,11 +1454,12 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
                                 throw new IgniteInterruptedException(e.getMessage());
                             }
 
+                            // Retry to send the request once more after some time.
                             send(singletonList(node), req);
 
-                            continue;
+                            break;
 
-                        case GridH2IndexRangeResponse.STATUS_ERROR:
+                        case STATUS_ERROR:
                             throw new CacheException(res.error());
 
                         default:

http://git-wip-us.apache.org/repos/asf/ignite/blob/9397c199/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index d8b1180..7b7711a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.query.h2.twostep;
 
 import java.lang.reflect.Field;
+import java.sql.Connection;
 import java.sql.ResultSet;
 import java.util.AbstractCollection;
 import java.util.ArrayList;
@@ -511,19 +512,22 @@ public class GridMapQueryExecutor {
                 }
             }
 
+            Connection conn = h2.connectionForSpace(mainCache);
+
+            // Here we enforce join order to have the same behavior on all the nodes.
+            h2.setupConnection(conn, distributedJoins, true);
+
             GridH2QueryContext.set(qctx);
 
             // qctx is set, we have to release reservations inside of it.
             reserved = null;
 
-            h2.enforceJoinOrder(true);
-
             try {
                 // Run queries.
                 int i = 0;
 
                 for (GridCacheSqlQuery qry : qrys) {
-                    ResultSet rs = h2.executeSqlQueryWithTimer(mainCache, h2.connectionForSpace(mainCache), qry.query(),
+                    ResultSet rs = h2.executeSqlQueryWithTimer(mainCache, conn, qry.query(),
                         F.asList(qry.parameters()), true);
 
                     if (ctx.event().isRecordable(EVT_CACHE_QUERY_EXECUTED)) {
@@ -559,8 +563,6 @@ public class GridMapQueryExecutor {
                 }
             }
             finally {
-                h2.enforceJoinOrder(false);
-
                 GridH2QueryContext.clear(distributedJoins);
 
                 if (!F.isEmpty(snapshotedTbls)) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/9397c199/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 6004629..fffb822 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -679,11 +679,11 @@ public class GridReduceQueryExecutor {
                     else {
                         UUID locNodeId = ctx.localNodeId();
 
+                        h2.setupConnection(r.conn, false, enforceJoinOrder);
+
                         GridH2QueryContext.set(new GridH2QueryContext(locNodeId, locNodeId, qryReqId, REDUCE)
                             .pageSize(r.pageSize).distributedJoins(false));
 
-                        h2.enforceJoinOrder(enforceJoinOrder);
-
                         try {
                             if (qry.explain())
                                 return explainPlan(r.conn, space, qry);
@@ -699,8 +699,6 @@ public class GridReduceQueryExecutor {
                             resIter = new Iter(res);
                         }
                         finally {
-                            h2.enforceJoinOrder(false);
-
                             GridH2QueryContext.clear(false);
                         }
                     }


[24/50] [abbrv] ignite git commit: Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-split2

Posted by sb...@apache.org.
Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-split2

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
#	modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java


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

Branch: refs/heads/ignite-1232
Commit: 1df11624aad98549472edc3507b6a9805b759d12
Parents: dddb817
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Dec 14 21:12:21 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Dec 14 21:12:21 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/binary/CacheObjectBinaryProcessorImpl.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1df11624/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
index 15dd088..5b74f0f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
@@ -457,10 +457,10 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
 
     /** {@inheritDoc} */
     @Override public String affinityField(String keyType) {
-        if (portableCtx == null)
+        if (binaryCtx == null)
             return null;
 
-        return portableCtx.affinityKeyFieldName(typeId(keyType));
+        return binaryCtx.affinityKeyFieldName(typeId(keyType));
     }
 
     /** {@inheritDoc} */


[44/50] [abbrv] ignite git commit: ignite-split2 - partially revert

Posted by sb...@apache.org.
ignite-split2 - partially revert


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

Branch: refs/heads/ignite-1232
Commit: e448d077e8660a424fa493ba4397ca2a617d1b20
Parents: b9b983a
Author: S.Vladykin <sv...@gridgain.com>
Authored: Sat Jan 9 15:26:47 2016 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Sat Jan 9 15:26:47 2016 +0300

----------------------------------------------------------------------
 .../processors/query/h2/twostep/GridReduceQueryExecutor.java   | 6 ++----
 1 file changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e448d077/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 3319797..409e26f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -412,13 +412,11 @@ public class GridReduceQueryExecutor {
     ) {
         String space = cctx.name();
 
-        Collection<ClusterNode> nodes = dataNodes(space, topVer);
+        Set<ClusterNode> nodes = new HashSet<>(dataNodes(space, topVer));
 
         if (F.isEmpty(nodes))
             throw new CacheException("Failed to find data nodes for cache: " + space);
 
-        nodes = nodes.size() > 4 ? new HashSet<>(nodes) : new ArrayList<>(nodes);
-
         if (!F.isEmpty(extraSpaces)) {
             for (String extraSpace : extraSpaces) {
                 GridCacheContext<?,?> extraCctx = cacheContext(extraSpace);
@@ -482,7 +480,7 @@ public class GridReduceQueryExecutor {
         if (!F.isEmpty(m)) {
             for (Map.Entry<IgniteProductVersion,Collection<ClusterNode>> entry : m.entrySet()) {
                 if (entry.getKey().compareTo(DISTRIBUTED_JOIN_SINCE) >= 0)
-                    break;
+                        break;
 
                 for (ClusterNode node : entry.getValue()) {
                     if (!node.isClient() && !node.isDaemon())


[27/50] [abbrv] ignite git commit: ignite-split2 - minor

Posted by sb...@apache.org.
ignite-split2 - minor


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

Branch: refs/heads/ignite-1232
Commit: 196346fce18ca0ad7fdbfa17ce0e386febb765e6
Parents: 084697b
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Dec 15 03:44:02 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Dec 15 03:44:02 2015 +0300

----------------------------------------------------------------------
 .../processors/query/h2/opt/GridH2QueryContext.java       | 10 ++++++++++
 .../internal/processors/query/h2/opt/GridH2TreeIndex.java |  6 +++---
 .../processors/query/h2/twostep/GridMapQueryExecutor.java |  2 ++
 .../processors/query/IgniteSqlSplitterSelfTest.java       |  3 +++
 4 files changed, 18 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/196346fc/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
index 8812086..c7d98c1 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
@@ -371,6 +371,16 @@ public class GridH2QueryContext {
     }
 
     /**
+     * @param nodeId Dead node ID.
+     */
+    public static void clearAfterDeadNode(UUID nodeId) {
+        for (Key key : qctxs.keySet()) {
+            if (key.nodeId.equals(nodeId))
+                doClear(key);
+        }
+    }
+
+    /**
      * Access current thread local query context (if it was set).
      *
      * @return Current thread local query context or {@code null} if the query runs outside of Ignite context.

http://git-wip-us.apache.org/repos/asf/ignite/blob/196346fc/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index 2677f07..eed21c4 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -1391,7 +1391,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
         private GridH2IndexRangeResponse awaitForResponse() {
             assert remainingRanges > 0;
 
-            for (int attempt = 0; attempt < 40; attempt++) {
+            for (int attempt = 0; attempt < 50; attempt++) {
                 GridH2IndexRangeResponse res;
 
                 try {
@@ -1446,11 +1446,11 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
                 }
 
                 if (!kernalContext().discovery().alive(node))
-                    throw new GridH2RetryException("Node left.");
+                    throw new GridH2RetryException("Node left: " + node);
             }
 
             // Attempts exceeded.
-            throw new CacheException("Failed to get index range from remote node.");
+            throw new CacheException("Failed to get index range from remote node, request timeout.");
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/196346fc/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index a88b3d9..6e1aef7 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -145,6 +145,8 @@ public class GridMapQueryExecutor {
             @Override public void onEvent(final Event evt) {
                 UUID nodeId = ((DiscoveryEvent)evt).eventNode().id();
 
+                GridH2QueryContext.clearAfterDeadNode(nodeId);
+
                 ConcurrentMap<Long,QueryResults> nodeRess = qryRess.remove(nodeId);
 
                 if (nodeRess == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/196346fc/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
index 31eb421..506be06 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
@@ -222,6 +222,9 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
 
             doTestDistributedJoins(c, 30, 100, 1000, false);
             doTestDistributedJoins(c, 30, 100, 1000, true);
+
+            doTestDistributedJoins(c, 100, 2000, 10, false);
+            doTestDistributedJoins(c, 100, 2000, 10, true);
         }
         finally {
             c.destroy();


[03/50] [abbrv] ignite git commit: Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-split2

Posted by sb...@apache.org.
Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-split2

# Conflicts:
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
#	modules/spring/pom.xml
#	parent/pom.xml


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

Branch: refs/heads/ignite-1232
Commit: 35e4b2098fc39ef4e1418748e37fac2fe848a420
Parents: f5c6370 171e9b1
Author: S.Vladykin <sv...@gridgain.com>
Authored: Fri Dec 4 08:05:50 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Fri Dec 4 08:05:50 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |  19 +
 assembly/release-fabric-base.xml                |   9 +
 .../config/binary/example-ignite-binary.xml     |  44 --
 examples/config/example-ignite.xml              |  10 +-
 examples/pom.xml                                |   7 +-
 examples/schema-import/pom.xml                  |   4 +-
 .../hibernate/CacheHibernatePersonStore.java    |   4 +-
 .../hibernate/CacheHibernateStoreExample.java   |   2 +-
 .../datagrid/store/hibernate/Person.hbm.xml     |   2 +-
 .../apache/ignite/examples/binary/Address.java  |  72 ---
 .../apache/ignite/examples/binary/Employee.java |  93 ----
 .../ignite/examples/binary/EmployeeKey.java     |  90 ----
 .../binary/ExampleBinaryNodeStartup.java        |  36 --
 .../ignite/examples/binary/Organization.java    |  93 ----
 .../examples/binary/OrganizationType.java       |  32 --
 ...ComputeClientBinaryTaskExecutionExample.java |  10 +-
 .../CacheClientBinaryPutGetExample.java         |  12 +-
 .../datagrid/CacheClientBinaryQueryExample.java |  28 +-
 .../ComputeFibonacciContinuationExample.java    |  15 +-
 .../computegrid/ComputeTaskMapExample.java      |   6 +-
 .../computegrid/ComputeTaskSplitExample.java    |   4 +-
 .../examples/datagrid/CacheQueryExample.java    | 161 ++-----
 .../ignite/examples/datagrid/store/Person.java  | 154 -------
 .../store/auto/CacheAutoStoreExample.java       |   4 +-
 .../auto/CacheAutoStoreLoadDataExample.java     |   4 +-
 .../datagrid/store/auto/CacheConfig.java        |   6 +-
 .../store/dummy/CacheDummyPersonStore.java      |  12 +-
 .../store/dummy/CacheDummyStoreExample.java     |   4 +-
 .../store/jdbc/CacheJdbcPersonStore.java        |  18 +-
 .../store/jdbc/CacheJdbcStoreExample.java       |   4 +-
 .../store/spring/CacheSpringPersonStore.java    |  10 +-
 .../store/spring/CacheSpringStoreExample.java   |   4 +-
 .../examples/messaging/MessagingExample.java    |  18 +-
 .../examples/misc/springbean/spring-bean.xml    |   7 -
 .../ignite/examples/model/Organization.java     |  62 +++
 .../apache/ignite/examples/model/Person.java    | 127 ++++++
 .../ignite/examples/model/binary/Address.java   |  72 +++
 .../ignite/examples/model/binary/Employee.java  |  93 ++++
 .../examples/model/binary/EmployeeKey.java      |  93 ++++
 .../examples/model/binary/Organization.java     |  93 ++++
 .../examples/model/binary/OrganizationType.java |  32 ++
 .../scalar/examples/ScalarCacheExample.scala    |  19 +-
 .../examples/ScalarCacheQueryExample.scala      |  66 +--
 .../testsuites/IgniteExamplesSelfTestSuite.java |   4 +-
 modules/aop/pom.xml                             |  14 +-
 modules/apache-license-gen/pom.xml              |   2 +-
 modules/aws/pom.xml                             |  20 +-
 modules/camel/pom.xml                           |  16 +-
 .../ignite/stream/camel/package-info.java       |  21 +
 .../ignite/stream/camel/package-info.java       |  21 +
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 .../ignite/codegen/MessageCodeGenerator.java    |   1 +
 modules/core/pom.xml                            |  28 +-
 .../java/org/apache/ignite/IgniteBinary.java    |  24 +-
 .../java/org/apache/ignite/IgniteCache.java     |   3 +-
 .../apache/ignite/binary/BinaryIdMapper.java    |   3 +-
 .../org/apache/ignite/binary/BinaryObject.java  |  17 +-
 .../org/apache/ignite/binary/BinaryType.java    |  16 +-
 .../ignite/binary/BinaryTypeConfiguration.java  |  21 +
 .../apache/ignite/cache/CacheInterceptor.java   |   2 +-
 .../ignite/cache/CacheKeyConfiguration.java     |  19 +
 .../apache/ignite/cache/CacheTypeMetadata.java  |   2 +
 .../rendezvous/RendezvousAffinityFunction.java  |   2 +-
 .../store/jdbc/CacheAbstractJdbcStore.java      |   2 +-
 .../cache/store/jdbc/CacheJdbcPojoStore.java    | 191 +++++---
 .../configuration/CacheConfiguration.java       |   6 +-
 .../configuration/IgniteConfiguration.java      |   4 +-
 .../internal/GridEventConsumeHandler.java       |   5 +
 .../internal/GridMessageListenHandler.java      |   5 +
 .../apache/ignite/internal/IgniteKernal.java    |  13 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |  10 +-
 .../ignite/internal/NodeStoppingException.java  |  35 ++
 .../managers/communication/GridIoManager.java   |  88 +---
 .../communication/GridIoMessageFactory.java     |   8 +-
 .../GridDeploymentPerVersionStore.java          |  12 +-
 .../discovery/GridDiscoveryManager.java         |  19 +-
 .../internal/portable/BinaryEnumObjectImpl.java | 311 +++++++++++++
 .../internal/portable/BinaryFieldAccessor.java  |  83 +++-
 .../internal/portable/BinaryFieldImpl.java      |   6 +-
 .../internal/portable/BinaryMarshaller.java     | 142 ++++++
 .../internal/portable/BinaryMetadata.java       |  16 +-
 .../internal/portable/BinaryObjectEx.java       | 225 +---------
 .../internal/portable/BinaryObjectExImpl.java   | 251 +++++++++++
 .../internal/portable/BinaryObjectImpl.java     |  15 +-
 .../portable/BinaryObjectOffheapImpl.java       |   2 +-
 .../internal/portable/BinaryReaderExImpl.java   |  21 +-
 .../internal/portable/BinaryTypeImpl.java       |  12 +-
 .../internal/portable/BinaryWriteMode.java      |   3 +
 .../internal/portable/BinaryWriterExImpl.java   |  34 +-
 .../portable/GridPortableMarshaller.java        |   3 +
 .../portable/PortableClassDescriptor.java       |  28 +-
 .../internal/portable/PortableContext.java      | 139 ++++--
 .../ignite/internal/portable/PortableUtils.java | 166 ++++++-
 .../builder/BinaryObjectBuilderImpl.java        |  25 +-
 .../portable/builder/PortableBuilderEnum.java   |   2 +-
 .../builder/PortableBuilderSerializer.java      |  14 +-
 .../builder/PortableEnumArrayLazyValue.java     |   2 +-
 .../builder/PortableObjectArrayLazyValue.java   |   2 +-
 .../CacheDefaultBinaryAffinityKeyMapper.java    |  62 +++
 .../processors/cache/CacheInvokeResult.java     |  14 +
 .../processors/cache/CacheLazyEntry.java        |   2 +-
 .../internal/processors/cache/CacheObject.java  |   3 +
 .../processors/cache/CacheObjectContext.java    |  23 +-
 .../processors/cache/GridCacheAdapter.java      |  12 +-
 .../cache/GridCacheConcurrentMap.java           |  68 +--
 .../processors/cache/GridCacheContext.java      |  79 +++-
 .../GridCacheDefaultAffinityKeyMapper.java      |   9 +-
 .../cache/GridCacheDeploymentManager.java       |   6 +-
 .../processors/cache/GridCacheEventManager.java |  42 +-
 .../processors/cache/GridCacheIoManager.java    |  10 +-
 .../processors/cache/GridCacheMapEntry.java     |  10 +-
 .../processors/cache/GridCacheMessage.java      |   3 +-
 .../processors/cache/GridCacheMvccManager.java  |   3 +-
 .../GridCachePartitionExchangeManager.java      |  40 +-
 .../processors/cache/GridCachePreloader.java    |  16 +-
 .../cache/GridCachePreloaderAdapter.java        |   7 +-
 .../processors/cache/GridCacheProcessor.java    |  18 +-
 .../processors/cache/GridCacheProxyImpl.java    |  12 +
 .../processors/cache/IgniteCacheProxy.java      |  38 +-
 .../processors/cache/IgniteInternalCache.java   |   9 +
 .../cache/affinity/GridCacheAffinityImpl.java   |   3 +-
 .../CacheDataStructuresManager.java             |  76 ++--
 .../distributed/dht/GridDhtCacheAdapter.java    |  15 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |  28 +-
 .../distributed/dht/GridDhtLockFuture.java      |  35 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |  13 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   2 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |  43 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |  15 +-
 .../distributed/dht/GridNoStorageCacheMap.java  |   4 +-
 .../dht/GridPartitionedSingleGetFuture.java     |  18 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  43 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |   6 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   3 +-
 .../colocated/GridDhtColocatedLockFuture.java   |  17 +-
 .../dht/preloader/GridDhtPartitionDemander.java |  71 +--
 .../dht/preloader/GridDhtPreloader.java         |  31 +-
 .../preloader/GridDhtPreloaderAssignments.java  |  19 +-
 .../distributed/near/GridNearCacheAdapter.java  |   4 +-
 .../distributed/near/GridNearGetFuture.java     |  16 +-
 .../distributed/near/GridNearGetRequest.java    |   3 +-
 .../distributed/near/GridNearLockFuture.java    |  24 +-
 ...arOptimisticSerializableTxPrepareFuture.java | 104 ++---
 .../near/GridNearOptimisticTxPrepareFuture.java | 191 ++++----
 ...ridNearOptimisticTxPrepareFutureAdapter.java |  70 +++
 .../near/GridNearSingleGetRequest.java          |  12 +-
 .../near/GridNearSingleGetResponse.java         |  13 +-
 .../near/GridNearTxFinishFuture.java            |  14 +-
 .../cache/local/GridLocalCacheEntry.java        |   6 +
 .../portable/CacheObjectBinaryProcessor.java    |  11 +-
 .../CacheObjectBinaryProcessorImpl.java         |  66 ++-
 .../portable/CacheObjectPortableContext.java    |   3 +-
 .../cache/portable/IgniteBinaryImpl.java        |  21 +-
 .../cache/query/GridCacheQueryManager.java      |  12 +-
 .../continuous/CacheContinuousQueryEntry.java   |  50 ++-
 .../continuous/CacheContinuousQueryEvent.java   |   6 +-
 .../continuous/CacheContinuousQueryHandler.java |  32 +-
 .../CacheContinuousQueryListener.java           |   5 +
 .../continuous/CacheContinuousQueryManager.java |  57 ++-
 .../cache/store/CacheOsStoreManager.java        |   2 +-
 .../cache/transactions/IgniteTxHandler.java     |  12 +-
 .../transactions/IgniteTxLocalAdapter.java      |  38 +-
 .../cache/transactions/IgniteTxManager.java     |  22 +-
 .../IgniteTxRemoteSingleStateImpl.java          |  16 +-
 .../cache/transactions/IgniteTxRemoteState.java |   5 +
 .../transactions/IgniteTxRemoteStateImpl.java   |  24 +
 .../processors/cacheobject/NoOpBinary.java      |  13 +-
 .../continuous/GridContinuousHandler.java       |   5 +
 .../continuous/GridContinuousProcessor.java     |  10 +-
 .../StartRoutineDiscoveryMessage.java           |  13 +-
 .../datastructures/CollocatedQueueItemKey.java  |  75 ++++
 .../datastructures/CollocatedSetItemKey.java    |  87 ++++
 .../datastructures/DataStructuresProcessor.java |  27 +-
 .../GridAtomicCacheQueueImpl.java               |   8 +-
 .../GridCacheInternalKeyImpl.java               |   2 +-
 .../datastructures/GridCacheQueueAdapter.java   |  30 +-
 .../datastructures/GridCacheQueueItemKey.java   |   9 +-
 .../datastructures/GridCacheSetImpl.java        |  37 +-
 .../datastructures/GridCacheSetItemKey.java     |  21 +-
 .../GridTransactionalCacheQueueImpl.java        |   2 +-
 .../processors/datastructures/QueueItemKey.java |  27 ++
 .../processors/datastructures/SetItemKey.java   |  36 ++
 .../platform/PlatformContextImpl.java           |  53 ++-
 .../cpp/PlatformCppConfigurationClosure.java    |   2 +-
 .../PlatformDotNetConfigurationClosure.java     |   2 +-
 .../transactions/PlatformTransactions.java      |   4 +-
 .../platform/utils/PlatformUtils.java           |   1 +
 .../processors/query/GridQueryProcessor.java    |  36 +-
 .../handlers/query/QueryCommandHandler.java     |   2 +-
 .../ignite/internal/util/IgniteUtils.java       |   8 +
 .../util/future/GridCompoundFuture.java         |   9 +-
 .../internal/util/future/GridFutureAdapter.java |   2 +-
 .../marshaller/portable/BinaryMarshaller.java   | 146 ------
 .../PlatformDotNetBinaryTypeConfiguration.java  |  23 +
 .../sharedfs/SharedFsCheckpointSpi.java         |  14 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  31 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   2 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   2 -
 .../core/src/main/resources/ignite.properties   |   2 +-
 ...heJdbcPojoStoreBinaryMarshallerSelfTest.java |  85 ++++
 ...JdbcPojoStorePortableMarshallerSelfTest.java |  85 ----
 .../internal/GridMultipleJobsSelfTest.java      |   7 +
 ...ectionLocalJobMultipleArgumentsSelfTest.java |  11 +-
 .../IgniteClientReconnectCacheTest.java         |  26 +-
 .../internal/portable/BinaryEnumsSelfTest.java  | 446 +++++++++++++++++++
 .../portable/BinaryFieldsAbstractSelfTest.java  |  13 +-
 .../portable/BinaryFieldsHeapSelfTest.java      |   4 +-
 .../portable/BinaryFieldsOffheapSelfTest.java   |   3 +-
 .../BinaryFooterOffsetsAbstractSelfTest.java    |   5 +-
 .../BinaryFooterOffsetsHeapSelfTest.java        |   4 +-
 .../BinaryFooterOffsetsOffheapSelfTest.java     |   3 +-
 .../portable/BinaryMarshallerSelfTest.java      | 306 ++++++++++---
 .../BinaryObjectBuilderAdditionalSelfTest.java  |   7 +-
 .../portable/BinaryObjectBuilderSelfTest.java   | 102 +++--
 .../GridPortableAffinityKeySelfTest.java        |  13 +-
 ...idPortableMarshallerCtxDisabledSelfTest.java |   1 -
 .../portable/GridPortableMetaDataSelfTest.java  |  11 +-
 .../portable/GridPortableWildcardsSelfTest.java |   1 -
 .../CachePutEventListenerErrorSelfTest.java     | 185 ++++++++
 ...cheAbstractFullApiMultithreadedSelfTest.java |   9 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  99 +++-
 .../cache/GridCacheClearSelfTest.java           |   9 +-
 .../cache/GridCacheConcurrentMapSelfTest.java   |   2 +-
 ...inodeUpdateNearEnabledNoBackupsSelfTest.java |   9 +-
 ...CacheMultinodeUpdateNearEnabledSelfTest.java |   5 +
 .../GridCacheOnCopyFlagAbstractSelfTest.java    |  14 +-
 .../processors/cache/GridCacheStopSelfTest.java | 143 +++++-
 .../cache/IgniteCacheAbstractTest.java          |   3 +
 ...niteCacheCopyOnReadDisabledAbstractTest.java |   2 +
 .../cache/IgniteCacheInvokeAbstractTest.java    | 369 +++++++++++----
 .../IgniteCacheStoreValueAbstractTest.java      | 111 +++--
 .../cache/IgniteCacheTxPreloadNoWriteTest.java  |   2 +-
 ...IgniteCacheAbstractExecutionContextTest.java |   2 +-
 ...eAbstractDataStructuresFailoverSelfTest.java |  39 +-
 .../GridCacheQueueApiSelfAbstractTest.java      |  18 +-
 .../GridCacheSetFailoverAbstractSelfTest.java   |   6 +-
 .../GridCachePartitionedQueueApiSelfTest.java   |   5 +
 ...dCachePartitionedQueueEntryMoveSelfTest.java |   2 +-
 .../IgnitePartitionedQueueNoBackupsTest.java    |  92 ++++
 .../GridCacheReplicatedQueueApiSelfTest.java    |   5 +
 .../CacheLockReleaseNodeLeaveTest.java          | 161 +++++++
 ...yMetadataUpdateChangingTopologySelfTest.java | 245 ++++++++++
 .../IgniteCacheSizeFailoverTest.java            |   7 +
 .../dht/GridCacheDhtInternalEntrySelfTest.java  | 203 ---------
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |  37 +-
 ...gniteAtomicLongChangingTopologySelfTest.java |  40 +-
 .../IgniteCacheCrossCacheTxFailoverTest.java    |   6 +
 .../IgniteCachePutRetryAbstractSelfTest.java    |   2 +
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |   3 +-
 .../near/GridCacheNearTxForceKeyTest.java       |   2 +-
 ...achePartitionedPreloadLifecycleSelfTest.java |  71 +--
 ...cingDelayedPartitionMapExchangeSelfTest.java |   9 +-
 .../GridCacheRebalancingAsyncSelfTest.java      |   3 +-
 .../GridCacheRebalancingSyncCheckDataTest.java  |  98 ++++
 .../GridCacheRebalancingSyncSelfTest.java       |  55 +--
 ...eRebalancingUnmarshallingFailedSelfTest.java |  19 +-
 .../replicated/GridReplicatedTxPreloadTest.java |   6 +
 ...CacheReplicatedPreloadLifecycleSelfTest.java | 100 +++--
 .../GridCacheEvictionLockUnlockSelfTest.java    |   2 +-
 .../random/RandomEvictionPolicySelfTest.java    |   8 +-
 .../GridCacheLocalIsolatedNodesSelfTest.java    |   6 +-
 ...naryObjectsAbstractDataStreamerSelfTest.java |   2 +-
 ...aryObjectsAbstractMultiThreadedSelfTest.java |  10 +-
 .../GridCacheBinaryObjectsAbstractSelfTest.java |   2 +-
 ...ntNodeBinaryObjectMetadataMultinodeTest.java |  10 +-
 ...CacheClientNodeBinaryObjectMetadataTest.java |   6 +-
 .../GridCachePortableStoreAbstractSelfTest.java |   2 +-
 ...ridPortableCacheEntryMemorySizeSelfTest.java |   2 +-
 ...leDuplicateIndexObjectsAbstractSelfTest.java |   2 +-
 .../DataStreamProcessorPortableSelfTest.java    |   2 +-
 .../GridDataStreamerImplSelfTest.java           |   2 +-
 ...ridCacheAffinityRoutingPortableSelfTest.java |   2 +-
 .../GridCacheMemoryModePortableSelfTest.java    |   2 +-
 ...acheOffHeapTieredAtomicPortableSelfTest.java |   2 +-
 ...eapTieredEvictionAtomicPortableSelfTest.java |   2 +-
 ...heOffHeapTieredEvictionPortableSelfTest.java |   2 +-
 .../GridCacheOffHeapTieredPortableSelfTest.java |   2 +-
 ...sNearPartitionedByteArrayValuesSelfTest.java |   2 +-
 ...sPartitionedOnlyByteArrayValuesSelfTest.java |   2 +-
 .../GridCacheWriteBehindStoreAbstractTest.java  |   2 +-
 .../cache/GridCacheCommandHandlerSelfTest.java  |   7 +-
 .../internal/util/nio/GridNioSelfTest.java      |  69 +--
 .../PlatformComputeBinarizableArgTask.java      |   8 +-
 .../platform/PlatformComputeEchoTask.java       |   4 +-
 .../session/GridSessionCheckpointSelfTest.java  |  16 +
 .../tcp/IgniteCacheSslStartStopSelfTest.java    |   1 +
 .../ignite/testframework/GridTestUtils.java     |  61 ++-
 .../config/GridTestProperties.java              |   3 +
 .../testframework/junits/GridAbstractTest.java  |  15 +
 .../ignite/testframework/junits/IgniteMock.java |  39 +-
 .../junits/IgniteTestResources.java             |  11 +
 ...ObjectsCacheDataStructuresSelfTestSuite.java |  37 ++
 ...BinaryObjectsCacheExpiryPolicyTestSuite.java |  38 ++
 ...gniteBinaryObjectsCacheRestartTestSuite.java |  37 ++
 .../IgniteBinaryObjectsCacheTestSuite2.java     |  37 ++
 .../IgniteBinaryObjectsCacheTestSuite3.java     |  37 ++
 .../IgniteBinaryObjectsCacheTestSuite4.java     |  37 ++
 ...IgniteBinaryObjectsComputeGridTestSuite.java |  37 ++
 .../IgniteCacheDataStructuresSelfTestSuite.java |   3 +
 .../IgniteCacheFailoverTestSuite3.java          |   6 -
 .../IgniteCacheFailoverTestSuiteSsl.java        |  41 ++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   7 +-
 .../testsuites/IgniteCacheTestSuite2.java       |   2 -
 .../testsuites/IgniteCacheTestSuite3.java       |   2 +
 .../IgnitePortableCacheFullApiTestSuite.java    |   2 +-
 .../IgnitePortableCacheTestSuite.java           |   3 +-
 .../IgnitePortableObjectsTestSuite.java         |   4 +
 modules/extdata/p2p/pom.xml                     |   2 +-
 .../extdata/uri/modules/uri-dependency/pom.xml  |   2 +-
 modules/extdata/uri/pom.xml                     |   4 +-
 modules/flume/README.txt                        |  44 +-
 modules/flume/pom.xml                           |  21 +-
 .../ignite/stream/flume/package-info.java       |  21 +
 .../ignite/stream/flume/package-info.java       |  21 +
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |  22 +-
 modules/hadoop/pom.xml                          |   4 +-
 modules/hibernate/pom.xml                       |  10 +-
 modules/indexing/pom.xml                        |  22 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  24 +-
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |  29 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |   2 +-
 .../cache/IgniteCacheNoClassQuerySelfTest.java  | 122 +++++
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |   7 -
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 .../IgnitePortableCacheQueryTestSuite.java      |   2 +-
 modules/jcl/pom.xml                             |  13 +-
 modules/jms11/README.txt                        |  29 ++
 modules/jms11/licenses/apache-2.0.txt           | 202 +++++++++
 modules/jms11/pom.xml                           |  22 +-
 .../ignite/stream/jms11/package-info.java       |  21 +
 .../ignite/stream/jms11/package-info.java       |  21 +
 modules/jta/pom.xml                             |  19 +-
 modules/kafka/README.txt                        |  32 ++
 modules/kafka/pom.xml                           |  18 +-
 .../ignite/stream/kafka/package-info.java       |  21 +
 .../ignite/stream/kafka/package-info.java       |  21 +
 modules/log4j/pom.xml                           |  12 +-
 modules/log4j2/pom.xml                          |  13 +-
 modules/mesos/pom.xml                           |   2 +-
 modules/mqtt/README.txt                         |  29 ++
 modules/mqtt/licenses/apache-2.0.txt            | 202 +++++++++
 modules/mqtt/pom.xml                            |  29 +-
 .../apache/ignite/stream/mqtt/package-info.java |  21 +
 .../apache/ignite/stream/mqtt/package-info.java |  21 +
 modules/osgi-karaf/README.txt                   |  18 +
 modules/osgi-karaf/licenses/apache-2.0.txt      | 202 +++++++++
 modules/osgi-karaf/pom.xml                      |  84 ++++
 .../osgi-karaf/src/main/resources/features.xml  | 327 ++++++++++++++
 modules/osgi-paxlogging/README.txt              |  12 +
 modules/osgi-paxlogging/licenses/apache-2.0.txt | 202 +++++++++
 modules/osgi-paxlogging/pom.xml                 |  69 +++
 modules/osgi/README.txt                         |  65 +++
 modules/osgi/licenses/apache-2.0.txt            | 202 +++++++++
 modules/osgi/pom.xml                            | 171 +++++++
 .../IgniteAbstractOsgiContextActivator.java     | 238 ++++++++++
 .../org/apache/ignite/osgi/IgniteOsgiUtils.java |  69 +++
 .../BundleDelegatingClassLoader.java            | 147 ++++++
 .../classloaders/ContainerSweepClassLoader.java | 134 ++++++
 .../OsgiClassLoadingStrategyType.java           |  29 ++
 .../ignite/osgi/AbstractIgniteKarafTest.java    | 109 +++++
 .../IgniteKarafFeaturesInstallationTest.java    | 100 +++++
 .../ignite/osgi/IgniteOsgiServiceTest.java      | 131 ++++++
 .../apache/ignite/osgi/IgniteOsgiTestSuite.java |  32 ++
 .../activators/BasicIgniteTestActivator.java    |  76 ++++
 .../ignite/osgi/activators/TestOsgiFlags.java   |  53 +++
 .../osgi/activators/TestOsgiFlagsImpl.java      |  83 ++++
 modules/platforms/README.txt                    |   6 +
 modules/platforms/cpp/common/configure.ac       |   2 +-
 modules/platforms/cpp/core-test/configure.ac    |   2 +-
 modules/platforms/cpp/core/configure.ac         |   2 +-
 .../impl/binary/binary_type_updater_impl.cpp    |   4 +-
 modules/platforms/cpp/examples/configure.ac     |   2 +-
 modules/platforms/cpp/ignite/configure.ac       |   2 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Properties/AssemblyInfo.cs                  |  12 +-
 .../Binary/BinaryBuilderSelfTest.cs             |  97 ++--
 .../Binary/BinarySelfTest.cs                    |  48 +-
 .../Cache/CacheAbstractTest.cs                  |  17 +-
 .../Compute/ComputeApiTest.cs                   |   6 +
 .../Config/Compute/compute-grid1.xml            |   8 +-
 .../Config/marshaller-explicit.xml              |   2 +-
 .../Config/native-client-test-cache-store.xml   |   1 +
 .../Examples/ExamplesTest.cs                    |   5 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Services/ServiceProxyTest.cs                |  22 +-
 .../Apache.Ignite.Core.csproj                   |   5 +-
 .../Binary/BinaryTypeConfiguration.cs           |  16 +-
 .../dotnet/Apache.Ignite.Core/Binary/IBinary.cs | 136 ++++++
 .../Apache.Ignite.Core/Binary/IBinaryObject.cs  |  23 +-
 .../Apache.Ignite.Core/Binary/IBinaryType.cs    |  13 +
 .../Apache.Ignite.Core/Binary/IIgniteBinary.cs  | 120 -----
 .../dotnet/Apache.Ignite.Core/IIgnite.cs        |   6 +-
 .../Apache.Ignite.Core/Impl/Binary/Binary.cs    | 216 +++++++++
 .../Impl/Binary/BinaryEnum.cs                   | 134 ++++++
 .../Impl/Binary/BinaryFullTypeDescriptor.cs     |  14 +-
 .../Impl/Binary/BinaryObject.cs                 |  29 ++
 .../Impl/Binary/BinaryObjectBuilder.cs          |  44 +-
 .../Impl/Binary/BinaryReader.cs                 |  51 ++-
 .../Binary/BinarySurrogateTypeDescriptor.cs     |   6 +
 .../Impl/Binary/BinarySystemHandlers.cs         |  34 +-
 .../Impl/Binary/BinaryUtils.cs                  |  96 ++--
 .../Impl/Binary/BinaryWriter.cs                 | 104 +++--
 .../Impl/Binary/IBinaryTypeDescriptor.cs        |  50 +--
 .../Impl/Binary/IgniteBinary.cs                 | 192 --------
 .../Impl/Binary/Marshaller.cs                   |  88 ++--
 .../Impl/Binary/Metadata/BinaryType.cs          | 184 ++++----
 .../Impl/Binary/Metadata/BinaryTypeHolder.cs    |  34 +-
 .../Binary/Structure/BinaryStructureTracker.cs  |   2 +-
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  11 +-
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |   2 +-
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |  11 +-
 .../Impl/Transactions/TransactionsImpl.cs       |   4 +-
 .../Properties/AssemblyInfo.cs                  |   6 +-
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |   4 +-
 .../Messaging/MessagingExample.cs               |   9 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../examples/Config/example-cache-query.xml     |   9 +-
 .../dotnet/examples/Config/example-cache.xml    |   9 +-
 modules/rest-http/pom.xml                       |  36 +-
 modules/scalar-2.10/pom.xml                     |  11 +-
 modules/scalar/pom.xml                          |  11 +-
 .../test/resources/spring-ping-pong-partner.xml |  18 -
 modules/schedule/pom.xml                        |  27 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |  14 +-
 modules/spark-2.10/pom.xml                      |   4 +-
 modules/spark/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |  27 +-
 .../ignite/cache/store/spring/package-info.java |  21 +
 modules/ssh/pom.xml                             |  21 +-
 modules/tools/pom.xml                           |   2 +-
 modules/twitter/pom.xml                         |  17 +-
 .../ignite/stream/twitter/package-info.java     |  21 +
 .../twitter/IgniteTwitterStreamerTestSuite.java |  19 +-
 .../ignite/stream/twitter/package-info.java     |  21 +
 modules/urideploy/pom.xml                       |  19 +-
 modules/visor-console-2.10/pom.xml              |   6 +-
 modules/visor-console/pom.xml                   |   4 +-
 modules/visor-plugins/pom.xml                   |   6 +-
 modules/web/pom.xml                             |  15 +-
 modules/yardstick/pom.xml                       |   6 +-
 ...IgniteTransactionalInvokeRetryBenchmark.java |   4 +-
 ...IgniteTransactionalWriteInvokeBenchmark.java |   6 +-
 modules/yarn/pom.xml                            |   8 +-
 .../yarn/IgniteApplicationMasterSelfTest.java   |  11 +-
 modules/zookeeper/README.txt                    |  29 ++
 modules/zookeeper/licenses/apache-2.0.txt       | 202 +++++++++
 modules/zookeeper/pom.xml                       |  16 +-
 .../discovery/tcp/ipfinder/zk/package-info.java |  21 +
 parent/pom.xml                                  | 132 +++++-
 pom.xml                                         |  46 +-
 455 files changed, 12593 insertions(+), 4066 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/35e4b209/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/35e4b209/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index 6858ca2,c9b8e27..e387c0a
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@@ -714,9 -715,14 +715,14 @@@ public class GridIoMessageFactory imple
  
                  break;
  
-             // [-3..118] - this
+             case 119:
+                 msg = new BinaryEnumObjectImpl();
+ 
+                 break;
+ 
+             // [-3..119] - this
              // [120..123] - DR
 -            // [-4..-22] - SQL
 +            // [-4..-28] - SQL
              default:
                  if (ext != null) {
                      for (MessageFactory factory : ext) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/35e4b209/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 59ea40f,bb5d230..980ff67
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@@ -69,10 -47,11 +69,10 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
  import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtUnreservedPartitionException;
  import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
- import org.apache.ignite.internal.processors.datastructures.GridCacheSetItemKey;
  import org.apache.ignite.internal.processors.datastructures.GridSetQueryPredicate;
+ import org.apache.ignite.internal.processors.datastructures.SetItemKey;
  import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter;
  import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 -import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
  import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
  import org.apache.ignite.internal.processors.query.GridQueryIndexType;
  import org.apache.ignite.internal.processors.query.GridQueryProcessor;

http://git-wip-us.apache.org/repos/asf/ignite/blob/35e4b209/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/35e4b209/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/35e4b209/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 9424bfc,1437a16..802a0fa
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@@ -290,15 -281,8 +291,11 @@@ public class IgniteH2Indexing implement
      /** */
      private volatile GridKernalContext ctx;
  
 +    /** */
 +    private final ConcurrentMap<String, GridH2Table> dataTables = new ConcurrentHashMap8<>();
 +
-     /** */
-     private final ThreadLocal<StatementCache> stmtCache = new ThreadLocal<StatementCache>() {
-         @Override protected StatementCache initialValue() {
-             return new StatementCache(PREPARED_STMT_CACHE_SIZE);
-         }
-     };
+     /** Statement cache. */
+     private final ConcurrentHashMap<Thread, StatementCache> stmtCache = new ConcurrentHashMap<>();
  
      /** */
      private final GridBoundedConcurrentLinkedHashMap<T3<String, String, Boolean>, TwoStepCachedQuery> twoStepCache =

http://git-wip-us.apache.org/repos/asf/ignite/blob/35e4b209/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/35e4b209/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePortableCacheQueryTestSuite.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/35e4b209/modules/spring/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/35e4b209/parent/pom.xml
----------------------------------------------------------------------
diff --cc parent/pom.xml
index 38979d1,4fd7156..b7c0964
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@@ -35,16 -35,87 +35,88 @@@
  
      <properties>
          <ignite.edition>fabric</ignite.edition>
 -
 -        <!-- Build parameters. -->
 +        <hadoop.version>2.4.1</hadoop.version>
 +        <spark.version>1.5.2</spark.version>
-         <h2.version>1.0-SNAPSHOT</h2.version>
 +        <spring.version>4.1.0.RELEASE</spring.version>
          <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
          <maven.build.timestamp.format>MMMM d yyyy</maven.build.timestamp.format>
          <doxygen.exec>doxygen</doxygen.exec>
          <git.exec>git</git.exec>
-         <jetty.version>9.2.11.v20150529</jetty.version>
+         <maven.bundle.plugin.version>2.5.4</maven.bundle.plugin.version>
          <javadoc.opts>-XDenableSunApiLintControl</javadoc.opts>
+ 
+         <!-- Dependency versions. -->
+         <activemq.version>5.12.0</activemq.version>
+         <aopalliance.bundle.version>1.0_6</aopalliance.bundle.version>
+         <asm.version>4.2</asm.version>
+         <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>
+         <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>
+         <commons.codec.version>1.6</commons.codec.version>
+         <commons.collections.version>3.2.1</commons.collections.version>
+         <commons.lang.version>2.6</commons.lang.version>
+         <cron4j.version>2.2.5</cron4j.version>
+         <curator.version>2.9.1</curator.version>
+         <ezmorph.bundle.version>1.0.6_1</ezmorph.bundle.version>
+         <ezmorph.version>1.0.6</ezmorph.version>
+         <flume.ng.version>1.6.0</flume.ng.version>
+         <guava.retrying.version>2.0.0</guava.retrying.version>
+         <guava.version>18.0</guava.version>
+         <guava14.version>14.0.1</guava14.version>
+         <guava16.version>16.0.1</guava16.version>
 -        <h2.version>1.3.175</h2.version>
++        <h2.version>1.0-SNAPSHOT</h2.version>
+         <hadoop.version>2.4.1</hadoop.version>
+         <httpclient.version>4.5.1</httpclient.version>
+         <httpcore.version>4.4.3</httpcore.version>
+         <jackson.version>1.9.13</jackson.version>
+         <javax.cache.bundle.version>1.0.0_1</javax.cache.bundle.version>
+         <javax.cache.version>1.0.0</javax.cache.version>
+         <jetty.version>9.2.11.v20150529</jetty.version>
+         <jms.spec.version>1.1.1</jms.spec.version>
+         <jsch.bundle.version>0.1.53_1</jsch.bundle.version>
+         <jsch.version>0.1.53</jsch.version>
+         <jsonlib.bundle.version>2.4_1</jsonlib.bundle.version>
+         <jsonlib.version>2.4</jsonlib.version>
+         <jtidy.version>r938</jtidy.version>
+         <kafka.bundle.version>0.8.2.1_1</kafka.bundle.version>
+         <kafka.clients.bundle.version>0.8.2.0_1</kafka.clients.bundle.version>
+         <kafka.clients.version>0.8.2.0</kafka.clients.version>
+         <kafka.version>0.8.2.1</kafka.version>
+         <kafka.version>0.8.2.1</kafka.version>
+         <karaf.version>4.0.2</karaf.version>
+         <lucene.bundle.version>3.5.0_1</lucene.bundle.version>
+         <lucene.version>3.5.0</lucene.version>
+         <oro.bundle.version>2.0.8_6</oro.bundle.version>
+         <osgi.core.version>5.0.0</osgi.core.version>
+         <osgi.enterprise.version>5.0.0</osgi.enterprise.version>
+         <paho.version>1.0.2</paho.version>
+         <scala210.jline.version>2.10.4</scala210.jline.version>
+         <scala210.library.version>2.10.4</scala210.library.version>
+         <scala211.library.version>2.11.7</scala211.library.version>
+         <slf4j.version>1.7.7</slf4j.version>
+         <slf4j16.version>1.6.4</slf4j16.version>
+         <spark.version>1.5.2</spark.version>
+         <spring.version>4.1.0.RELEASE</spring.version>
+         <spring41.osgi.feature.version>4.1.7.RELEASE_1</spring41.osgi.feature.version>
+         <tomcat.version>8.0.23</tomcat.version>
+         <twitter.hbc.version>2.2.0</twitter.hbc.version>
+         <yammer.metrics.annotation.version>2.2.0</yammer.metrics.annotation.version>
+         <yammer.metrics.core.version>2.2.0</yammer.metrics.core.version>
+         <zkclient.version>0.5</zkclient.version>
+         <zookeeper.version>3.4.6</zookeeper.version>
+ 
+         <!-- OSGI Manifest generation default property values -->
+         <osgi.import.package>*</osgi.import.package>
+         <osgi.export.package>{local-packages}</osgi.export.package>
+         <osgi.private.package></osgi.private.package>
+         <osgi.embed.dependency></osgi.embed.dependency>
+         <osgi.embed.transitive>false</osgi.embed.transitive>
+         <osgi.fail.ok>false</osgi.fail.ok>
+ 
      </properties>
  
      <groupId>org.apache.ignite</groupId>


[36/50] [abbrv] ignite git commit: ignite-split2 - fixes after merge

Posted by sb...@apache.org.
ignite-split2 - fixes after merge


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

Branch: refs/heads/ignite-1232
Commit: 2847880c29f8b6fc4603c31ad3d75d852b037a75
Parents: 5303fb8
Author: S.Vladykin <sv...@gridgain.com>
Authored: Sat Dec 26 08:15:58 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Sat Dec 26 08:15:58 2015 +0300

----------------------------------------------------------------------
 .../cache/query/GridCacheTwoStepQuery.java      | 28 ++++++++----
 .../processors/query/h2/IgniteH2Indexing.java   | 16 +++++--
 .../query/h2/sql/GridSqlQuerySplitter.java      | 47 ++++++++++----------
 .../h2/twostep/GridReduceQueryExecutor.java     |  2 +-
 .../IgniteBinaryCacheQueryTestSuite.java        | 11 -----
 .../IgniteCacheQuerySelfTestSuite.java          |  5 ---
 6 files changed, 57 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2847880c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
index dfc84b8..6238ca2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache.query;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Set;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -45,7 +46,10 @@ public class GridCacheTwoStepQuery {
     private boolean explain;
 
     /** */
-    private Set<String> spaces;
+    private Collection<String> spaces;
+
+    /** */
+    private Set<String> schemas;
 
     /** */
     private Set<String> tbls;
@@ -57,11 +61,11 @@ public class GridCacheTwoStepQuery {
     private boolean skipMergeTbl;
 
     /**
-     * @param spaces All spaces accessed in query.
-     * @param tbls Tables.
+     * @param schemas Schema names in query.
+     * @param tbls Tables in query.
      */
-    public GridCacheTwoStepQuery(Set<String> spaces, Set<String> tbls) {
-        this.spaces = spaces;
+    public GridCacheTwoStepQuery(Set<String> schemas, Set<String> tbls) {
+        this.schemas = schemas;
         this.tbls = tbls;
     }
 
@@ -160,26 +164,34 @@ public class GridCacheTwoStepQuery {
     /**
      * @return Spaces.
      */
-    public Set<String> spaces() {
+    public Collection<String> spaces() {
         return spaces;
     }
 
     /**
      * @param spaces Spaces.
      */
-    public void spaces(Set<String> spaces) {
+    public void spaces(Collection<String> spaces) {
         this.spaces = spaces;
     }
 
     /**
+     * @return Schemas.
+     */
+    public Set<String> schemas() {
+        return schemas;
+    }
+
+    /**
      * @param args New arguments to copy with.
      * @return Copy.
      */
     public GridCacheTwoStepQuery copy(Object[] args) {
         assert !explain;
 
-        GridCacheTwoStepQuery cp = new GridCacheTwoStepQuery(spaces, tbls);
+        GridCacheTwoStepQuery cp = new GridCacheTwoStepQuery(schemas, tbls);
 
+        cp.spaces = spaces;
         cp.rdc = rdc.copy(args);
         cp.skipMergeTbl = skipMergeTbl;
         cp.pageSize = pageSize;

http://git-wip-us.apache.org/repos/asf/ignite/blob/2847880c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 90dcc88..58026ca 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -953,7 +953,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         String sql = generateQuery(qry, tbl);
 
-        Connection conn = connectionForThread(tbl.schema());
+        Connection conn = connectionForThread(tbl.schemaName());
 
         initLocalQueryContext(conn, false, filter);
 
@@ -1105,6 +1105,16 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 twoStepQry = GridSqlQuerySplitter.split((JdbcPreparedStatement)stmt, qry.getArgs(), groupByCollocated,
                     distributedJoins);
 
+                // Setup spaces from schemas.
+                if (!twoStepQry.schemas().isEmpty()) {
+                    Collection<String> spaces = new ArrayList<>(twoStepQry.schemas().size());
+
+                    for (String schema : twoStepQry.schemas())
+                        spaces.add(space(schema));
+
+                    twoStepQry.spaces(spaces);
+                }
+
                 meta = meta(stmt.getMetaData());
             }
             catch (IgniteCheckedException e) {
@@ -1701,7 +1711,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         throws IgniteCheckedException {
         String schema = schemaNameFromCacheConf(ccfg);
 
-        if (schemas.putIfAbsent(schema, new Schema(ccfg.getName(), cctx, ccfg)) != null)
+        if (schemas.putIfAbsent(schema, new Schema(ccfg.getName(), schema, cctx, ccfg)) != null)
             throw new IgniteCheckedException("Cache already registered: " + U.maskName(ccfg.getName()));
 
         space2schema.put(emptyIfNull(ccfg.getName()), schema);
@@ -2460,7 +2470,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
          * @param cctx Cache context.
          * @param ccfg Cache configuration.
          */
-        private Schema(@Nullable String spaceName, GridCacheContext<?,?> cctx, CacheConfiguration<?,?> ccfg) {
+        private Schema(String spaceName, String schemaName, GridCacheContext<?,?> cctx, CacheConfiguration<?,?> ccfg) {
             this.spaceName = spaceName;
             this.cctx = cctx;
             this.schemaName = schemaName;

http://git-wip-us.apache.org/repos/asf/ignite/blob/2847880c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
index 08f12d9..a6781bd 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
@@ -26,7 +26,6 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery;
 import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
-import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgnitePredicate;
@@ -170,7 +169,7 @@ public class GridSqlQuerySplitter {
         qry = collectAllTables(qry, schemas, tbls);
 
         // Build resulting two step query.
-        GridCacheTwoStepQuery res = new GridCacheTwoStepQuery(spaces, tbls);
+        GridCacheTwoStepQuery res = new GridCacheTwoStepQuery(schemas, tbls);
 
         // Map query will be direct reference to the original query AST.
         // Thus all the modifications will be performed on the original AST, so we should be careful when
@@ -280,7 +279,7 @@ public class GridSqlQuerySplitter {
      * @param collocatedGroupBy Whether the query has collocated GROUP BY keys.
      * @return Reduce query for the given map query.
      */
-    public static GridCacheSqlQuery split(GridCacheTwoStepQuery res, int splitIdx, final GridSqlSelect mapQry,
+    private static GridCacheSqlQuery split(GridCacheTwoStepQuery res, int splitIdx, final GridSqlSelect mapQry,
         Object[] params, boolean collocatedGroupBy) {
         final boolean explain = mapQry.explain();
 
@@ -436,27 +435,26 @@ public class GridSqlQuerySplitter {
 
     /**
      * @param qry Query.
-     * @param spaces Space names.
+     * @param schemas Schema names.
      * @param tbls Tables.
-     * @param schemas Shemas' names.
      * @return Query.
      */
-    private static GridSqlQuery collectAllTables(GridSqlQuery qry, Set<String> spaces, Set<String> tbls) {
+    private static GridSqlQuery collectAllTables(GridSqlQuery qry, Set<String> schemas, Set<String> tbls) {
         if (qry instanceof GridSqlUnion) {
             GridSqlUnion union = (GridSqlUnion)qry;
 
-            collectAllTables(union.left(), spaces, tbls);
-            collectAllTables(union.right(), spaces, tbls);
+            collectAllTables(union.left(), schemas, tbls);
+            collectAllTables(union.right(), schemas, tbls);
         }
         else {
             GridSqlSelect select = (GridSqlSelect)qry;
 
-            collectAllTablesInFrom(select.from(), spaces, tbls);
+            collectAllTablesInFrom(select.from(), schemas, tbls);
 
             for (GridSqlElement el : select.columns(false))
-                collectAllTablesInSubqueries(el, spaces, tbls);
+                collectAllTablesInSubqueries(el, schemas, tbls);
 
-            collectAllTablesInSubqueries(select.where(), spaces, tbls);
+            collectAllTablesInSubqueries(select.where(), schemas, tbls);
         }
 
         return qry;
@@ -464,11 +462,10 @@ public class GridSqlQuerySplitter {
 
     /**
      * @param from From element.
-     * @param spaces Space names.
+     * @param schemas Schema names.
      * @param tbls Tables.
-     * @param schemas Shemas' names.
      */
-    private static void collectAllTablesInFrom(GridSqlElement from, final Set<String> spaces, final Set<String> tbls) {
+    private static void collectAllTablesInFrom(GridSqlElement from, final Set<String> schemas, final Set<String> tbls) {
         findTablesInFrom(from, new IgnitePredicate<GridSqlElement>() {
             @Override public boolean apply(GridSqlElement el) {
                 if (el instanceof GridSqlTable) {
@@ -476,14 +473,16 @@ public class GridSqlQuerySplitter {
 
                     String schema = tbl.schema();
 
-                    if (schema != null && spaces != null)
-                        spaces.add(IgniteH2Indexing.space(schema));
+                    boolean addSchema = tbls == null;
 
                     if (tbls != null)
-                        tbls.add(tbl.dataTable().identifier());
+                        addSchema = tbls.add(tbl.dataTable().identifier());
+
+                    if (addSchema && schema != null && schemas != null)
+                        schemas.add(schema);
                 }
                 else if (el instanceof GridSqlSubquery)
-                    collectAllTables(((GridSqlSubquery)el).select(), spaces, tbls);
+                    collectAllTables(((GridSqlSubquery)el).select(), schemas, tbls);
 
                 return false;
             }
@@ -524,13 +523,13 @@ public class GridSqlQuerySplitter {
     }
 
     /**
-     * Searches spaces in subqueries in SELECT and WHERE clauses.
+     * Searches schema names and tables in subqueries in SELECT and WHERE clauses.
+     *
      * @param el Element.
-     * @param spaces Space names.
+     * @param schemas Schema names.
      * @param tbls Tables.
-     * @param schemas Schemas' names.
      */
-    private static void collectAllTablesInSubqueries(GridSqlElement el, Set<String> spaces, Set<String> tbls) {
+    private static void collectAllTablesInSubqueries(GridSqlElement el, Set<String> schemas, Set<String> tbls) {
         if (el == null)
             return;
 
@@ -538,10 +537,10 @@ public class GridSqlQuerySplitter {
 
         if (el instanceof GridSqlOperation || el instanceof GridSqlFunction) {
             for (GridSqlElement child : el)
-                collectAllTablesInSubqueries(child, spaces, tbls);
+                collectAllTablesInSubqueries(child, schemas, tbls);
         }
         else if (el instanceof GridSqlSubquery)
-            collectAllTables(((GridSqlSubquery)el).select(), spaces, tbls);
+            collectAllTables(((GridSqlSubquery)el).select(), schemas, tbls);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/2847880c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index fffb822..a5fca5d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -1043,7 +1043,7 @@ public class GridReduceQueryExecutor {
      * @param allSpaces All spaces.
      * @return List of all extra spaces or {@code null} if none.
      */
-    private List<String> extraSpaces(String mainSpace, Set<String> allSpaces) {
+    private List<String> extraSpaces(String mainSpace, Collection<String> allSpaces) {
         if (F.isEmpty(allSpaces) || (allSpaces.size() == 1 && allSpaces.contains(mainSpace)))
             return null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2847880c/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
index de94d53..2e0e4e2 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
@@ -32,7 +32,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexDisabledSe
 import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexingDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheQueryInternalKeysSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheQuerySerializationSelfTest;
-import org.apache.ignite.internal.processors.cache.GridCacheReduceQueryMultithreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteBinaryWrappedObjectFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheBinaryObjectsScanSelfTest;
@@ -95,10 +94,6 @@ import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheCon
 import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientReconnectTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientTxReconnectTest;
-import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryAtomicSelfTest;
-import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryLocalSelfTest;
-import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryPartitionedSelfTest;
-import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryReplicatedSelfTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSplitterSelfTest;
 import org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest;
 import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest;
@@ -200,12 +195,6 @@ public class IgniteBinaryCacheQueryTestSuite extends TestSuite {
         suite.addTestSuite(CacheContinuousQueryFailoverTxSelfTest.class);
         suite.addTestSuite(CacheContinuousQueryFailoverTxReplicatedSelfTest.class);
 
-        // Reduce fields queries.
-        suite.addTestSuite(GridCacheReduceFieldsQueryLocalSelfTest.class);
-        suite.addTestSuite(GridCacheReduceFieldsQueryPartitionedSelfTest.class);
-        suite.addTestSuite(GridCacheReduceFieldsQueryAtomicSelfTest.class);
-        suite.addTestSuite(GridCacheReduceFieldsQueryReplicatedSelfTest.class);
-
         suite.addTestSuite(GridCacheQueryIndexingDisabledSelfTest.class);
 
         //Should be adjusted. Not ready to be used with BinaryMarshaller.

http://git-wip-us.apache.org/repos/asf/ignite/blob/2847880c/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 5bef70b..0d099f2 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -29,7 +29,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexDisabledSe
 import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexingDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheQueryInternalKeysSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheQuerySerializationSelfTest;
-import org.apache.ignite.internal.processors.cache.GridCacheReduceQueryMultithreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteBinaryWrappedObjectFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheCollocatedQuerySelfTest;
@@ -90,10 +89,6 @@ import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheCon
 import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientReconnectTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientTxReconnectTest;
-import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryAtomicSelfTest;
-import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryLocalSelfTest;
-import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryPartitionedSelfTest;
-import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryReplicatedSelfTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSchemaIndexingTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSplitterSelfTest;
 import org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest;


[23/50] [abbrv] ignite git commit: Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-split2

Posted by sb...@apache.org.
Merge branch 'ignite-1.5' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-split2

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
#	modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java


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

Branch: refs/heads/ignite-1232
Commit: dddb817bb77ca263f76291517466541bc1238c0c
Parents: 16242b7 a3a8138
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Dec 14 21:12:05 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Dec 14 21:12:05 2015 +0300

----------------------------------------------------------------------
 .gitignore                                      |    1 +
 assembly/docfiles/ignite_logo.png               |  Bin 0 -> 24315 bytes
 assembly/release-fabric-base.xml                |   31 +-
 bin/ignite.bat                                  |    7 +-
 bin/include/parseargs.bat                       |    4 +-
 examples/pom.xml                                |    2 +-
 examples/schema-import/README.txt               |    2 +-
 examples/schema-import/pom.xml                  |    2 +-
 .../java/org/apache/ignite/schema/Demo.java     |    6 +-
 ...ComputeClientBinaryTaskExecutionExample.java |    4 +-
 .../CacheClientBinaryPutGetExample.java         |    8 +-
 .../datagrid/CacheClientBinaryQueryExample.java |   15 +-
 .../store/auto/CacheBinaryAutoStoreExample.java |   38 +-
 .../examples/datagrid/CacheQueryExample.java    |    6 +
 .../datagrid/CacheTransactionExample.java       |    4 +-
 .../starschema/CacheStarSchemaExample.java      |    9 +-
 .../store/auto/CacheAutoStoreExample.java       |   81 +-
 .../messaging/MessagingPingPongExample.java     |    4 -
 .../ignite/examples/model/Organization.java     |   16 +-
 .../apache/ignite/examples/model/Person.java    |   40 +-
 .../ignite/examples/util/DbH2ServerStartup.java |   31 +-
 .../examples/CacheClientBinaryExampleTest.java  |   46 +
 .../CacheClientPortableExampleTest.java         |   46 -
 .../ComputeClientBinaryExampleTest.java         |   37 +
 .../ComputeClientPortableExampleTest.java       |   37 -
 .../testsuites/IgniteExamplesSelfTestSuite.java |   15 +-
 .../examples/MessagingExamplesSelfTest.java     |    2 -
 modules/aop/pom.xml                             |    2 +-
 .../apache/ignite/gridify/AbstractAopTest.java  |   32 +-
 modules/apache-license-gen/pom.xml              |    2 +-
 modules/aws/pom.xml                             |    2 +-
 modules/camel/pom.xml                           |    2 +-
 modules/clients/pom.xml                         |   62 +-
 .../client/impl/ClientCacheFlagsCodecTest.java  |    8 +-
 .../src/test/resources/spring-server-node.xml   |    8 +-
 .../test/resources/spring-server-ssl-node.xml   |    8 +-
 modules/cloud/pom.xml                           |    2 +-
 modules/codegen/pom.xml                         |    2 +-
 .../ignite/codegen/MessageCodeGenerator.java    |    1 +
 modules/core/pom.xml                            |   10 +-
 .../java/org/apache/ignite/IgniteBinary.java    |  124 +-
 .../java/org/apache/ignite/IgniteCache.java     |    8 +-
 .../main/java/org/apache/ignite/Ignition.java   |   18 +-
 .../ignite/binary/BinaryCollectionFactory.java  |   33 +
 .../apache/ignite/binary/BinaryMapFactory.java  |   33 +
 .../apache/ignite/binary/BinaryRawReader.java   |    8 +-
 .../org/apache/ignite/binary/BinaryReader.java  |   13 +-
 .../org/apache/ignite/cache/QueryEntity.java    |    3 +-
 .../affinity/AffinityNodeHashResolver.java      |    6 +-
 .../store/jdbc/CacheAbstractJdbcStore.java      |    5 +-
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |    2 +-
 .../configuration/BinaryConfiguration.java      |    4 +-
 .../configuration/CacheConfiguration.java       |    5 +-
 .../configuration/IgniteConfiguration.java      |    4 +-
 .../configuration/TransactionConfiguration.java |   47 +-
 .../apache/ignite/internal/GridComponent.java   |    3 +-
 .../ignite/internal/GridKernalContextImpl.java  |    2 +-
 .../ignite/internal/GridPluginComponent.java    |    4 +-
 .../apache/ignite/internal/IgniteKernal.java    |   52 +-
 .../ignite/internal/IgniteNodeAttributes.java   |    4 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   65 +-
 .../ignite/internal/MarshallerContextImpl.java  |   23 +-
 .../binary/BinaryCachingMetadataHandler.java    |   70 +
 .../internal/binary/BinaryClassDescriptor.java  |  813 ++++
 .../ignite/internal/binary/BinaryContext.java   | 1102 +++++
 .../ignite/internal/binary/BinaryEnumCache.java |   69 +
 .../internal/binary/BinaryEnumObjectImpl.java   |  311 ++
 .../internal/binary/BinaryFieldAccessor.java    |  856 ++++
 .../ignite/internal/binary/BinaryFieldImpl.java |  116 +
 .../internal/binary/BinaryInternalIdMapper.java |  161 +
 .../internal/binary/BinaryMarshaller.java       |  142 +
 .../ignite/internal/binary/BinaryMetadata.java  |  263 ++
 .../binary/BinaryMetadataCollector.java         |  277 ++
 .../internal/binary/BinaryMetadataHandler.java  |   44 +
 .../binary/BinaryNoopMetadataHandler.java       |   53 +
 .../ignite/internal/binary/BinaryObjectEx.java  |   30 +
 .../internal/binary/BinaryObjectExImpl.java     |  251 ++
 .../internal/binary/BinaryObjectImpl.java       |  569 +++
 .../binary/BinaryObjectOffheapImpl.java         |  417 ++
 .../internal/binary/BinaryPositionReadable.java |   47 +
 .../internal/binary/BinaryPrimitives.java       |  382 ++
 .../internal/binary/BinaryRawReaderEx.java      |   33 +
 .../internal/binary/BinaryRawWriterEx.java      |   60 +
 .../internal/binary/BinaryReaderExImpl.java     | 2028 +++++++++
 .../internal/binary/BinaryReaderHandles.java    |  108 +
 .../binary/BinaryReaderHandlesHolder.java       |   46 +
 .../binary/BinaryReaderHandlesHolderImpl.java   |   44 +
 .../ignite/internal/binary/BinarySchema.java    |  494 +++
 .../internal/binary/BinarySchemaRegistry.java   |  172 +
 .../binary/BinaryThreadLocalContext.java        |   69 +
 .../ignite/internal/binary/BinaryTypeImpl.java  |   93 +
 .../ignite/internal/binary/BinaryUtils.java     | 1864 ++++++++
 .../ignite/internal/binary/BinaryWriteMode.java |  178 +
 .../internal/binary/BinaryWriterExImpl.java     | 1768 ++++++++
 .../internal/binary/BinaryWriterHandles.java    |  101 +
 .../binary/BinaryWriterSchemaHolder.java        |  148 +
 .../internal/binary/GridBinaryMarshaller.java   |  286 ++
 .../binary/builder/BinaryAbstractLazyValue.java |   57 +
 .../binary/builder/BinaryBuilderEnum.java       |  115 +
 .../binary/builder/BinaryBuilderReader.java     |  846 ++++
 .../BinaryBuilderSerializationAware.java        |   31 +
 .../binary/builder/BinaryBuilderSerializer.java |  217 +
 .../builder/BinaryEnumArrayLazyValue.java       |  113 +
 .../binary/builder/BinaryLazyArrayList.java     |  167 +
 .../binary/builder/BinaryLazyLinkedList.java    |  218 +
 .../internal/binary/builder/BinaryLazyMap.java  |  221 +
 .../internal/binary/builder/BinaryLazySet.java  |   92 +
 .../binary/builder/BinaryLazyValue.java         |   28 +
 .../builder/BinaryModifiableLazyValue.java      |   52 +
 .../builder/BinaryObjectArrayLazyValue.java     |   90 +
 .../binary/builder/BinaryObjectBuilderImpl.java |  572 +++
 .../binary/builder/BinaryPlainBinaryObject.java |   53 +
 .../binary/builder/BinaryPlainLazyValue.java    |   49 +
 .../binary/builder/BinaryValueWithType.java     |   76 +
 .../internal/binary/builder/package-info.java   |   22 +
 .../ignite/internal/binary/package-info.java    |   22 +
 .../streams/BinaryAbstractInputStream.java      |  379 ++
 .../streams/BinaryAbstractOutputStream.java     |  347 ++
 .../binary/streams/BinaryAbstractStream.java    |   80 +
 .../binary/streams/BinaryHeapInputStream.java   |  166 +
 .../binary/streams/BinaryHeapOutputStream.java  |  176 +
 .../binary/streams/BinaryInputStream.java       |  162 +
 .../binary/streams/BinaryMemoryAllocator.java   |   57 +
 .../streams/BinaryMemoryAllocatorChunk.java     |  117 +
 .../streams/BinaryOffheapInputStream.java       |  144 +
 .../streams/BinaryOffheapOutputStream.java      |  222 +
 .../binary/streams/BinaryOutputStream.java      |  259 ++
 .../internal/binary/streams/BinaryStream.java   |   53 +
 .../internal/binary/streams/package-info.java   |   22 +
 .../internal/client/GridClientCacheFlag.java    |   10 +-
 .../internal/client/GridClientCompute.java      |    4 +-
 .../client/GridClientConfiguration.java         |    4 +-
 .../client/impl/GridClientComputeImpl.java      |   20 +-
 .../impl/connection/GridClientConnection.java   |    6 +-
 .../GridClientConnectionManagerAdapter.java     |    6 +-
 .../connection/GridClientNioTcpConnection.java  |   34 +-
 .../internal/direct/DirectMessageWriter.java    |  108 +-
 .../internal/managers/GridManagerAdapter.java   |    5 +-
 .../managers/communication/GridIoManager.java   |    3 +-
 .../communication/GridIoMessageFactory.java     |   12 +-
 .../deployment/GridDeploymentManager.java       |    5 +-
 .../portable/BinaryCachingMetadataHandler.java  |   70 -
 .../internal/portable/BinaryEnumCache.java      |   69 -
 .../internal/portable/BinaryEnumObjectImpl.java |  311 --
 .../internal/portable/BinaryFieldAccessor.java  |  866 ----
 .../internal/portable/BinaryFieldImpl.java      |  116 -
 .../portable/BinaryInternalIdMapper.java        |  161 -
 .../internal/portable/BinaryMarshaller.java     |  142 -
 .../internal/portable/BinaryMetadata.java       |  180 -
 .../portable/BinaryMetadataCollector.java       |  277 --
 .../portable/BinaryMetadataHandler.java         |   44 -
 .../portable/BinaryNoopMetadataHandler.java     |   53 -
 .../internal/portable/BinaryObjectEx.java       |   30 -
 .../internal/portable/BinaryObjectExImpl.java   |  251 --
 .../internal/portable/BinaryObjectImpl.java     |  580 ---
 .../portable/BinaryObjectOffheapImpl.java       |  429 --
 .../internal/portable/BinaryRawReaderEx.java    |   33 -
 .../internal/portable/BinaryRawWriterEx.java    |   60 -
 .../internal/portable/BinaryReaderExImpl.java   | 2020 ---------
 .../internal/portable/BinaryReaderHandles.java  |  108 -
 .../portable/BinaryReaderHandlesHolder.java     |   46 -
 .../portable/BinaryReaderHandlesHolderImpl.java |   44 -
 .../portable/BinaryThreadLocalContext.java      |   69 -
 .../internal/portable/BinaryTypeImpl.java       |   93 -
 .../internal/portable/BinaryWriteMode.java      |  181 -
 .../internal/portable/BinaryWriterExImpl.java   | 1833 --------
 .../internal/portable/BinaryWriterHandles.java  |  101 -
 .../portable/BinaryWriterSchemaHolder.java      |  148 -
 .../portable/GridPortableMarshaller.java        |  307 --
 .../portable/PortableClassDescriptor.java       |  814 ----
 .../internal/portable/PortableContext.java      | 1112 -----
 .../portable/PortablePositionReadable.java      |   47 -
 .../internal/portable/PortablePrimitives.java   |  382 --
 .../internal/portable/PortableSchema.java       |  466 --
 .../portable/PortableSchemaRegistry.java        |  172 -
 .../ignite/internal/portable/PortableUtils.java | 1979 ---------
 .../builder/BinaryObjectBuilderImpl.java        |  580 ---
 .../builder/PortableAbstractLazyValue.java      |   57 -
 .../portable/builder/PortableBuilderEnum.java   |  116 -
 .../portable/builder/PortableBuilderReader.java |  859 ----
 .../PortableBuilderSerializationAware.java      |   31 -
 .../builder/PortableBuilderSerializer.java      |  222 -
 .../builder/PortableEnumArrayLazyValue.java     |  114 -
 .../portable/builder/PortableLazyArrayList.java |  166 -
 .../builder/PortableLazyLinkedList.java         |  217 -
 .../portable/builder/PortableLazyMap.java       |  220 -
 .../portable/builder/PortableLazyMapEntry.java  |   68 -
 .../portable/builder/PortableLazySet.java       |   92 -
 .../portable/builder/PortableLazyValue.java     |   28 -
 .../builder/PortableModifiableLazyValue.java    |   52 -
 .../builder/PortableObjectArrayLazyValue.java   |   91 -
 .../builder/PortablePlainLazyValue.java         |   49 -
 .../builder/PortablePlainPortableObject.java    |   53 -
 .../portable/builder/PortableValueWithType.java |   76 -
 .../internal/portable/builder/package-info.java |   22 -
 .../ignite/internal/portable/package-info.java  |   22 -
 .../streams/PortableAbstractInputStream.java    |  379 --
 .../streams/PortableAbstractOutputStream.java   |  347 --
 .../streams/PortableAbstractStream.java         |   80 -
 .../streams/PortableHeapInputStream.java        |  166 -
 .../streams/PortableHeapOutputStream.java       |  176 -
 .../portable/streams/PortableInputStream.java   |  162 -
 .../streams/PortableMemoryAllocator.java        |   57 -
 .../streams/PortableMemoryAllocatorChunk.java   |  117 -
 .../streams/PortableOffheapInputStream.java     |  144 -
 .../streams/PortableOffheapOutputStream.java    |  222 -
 .../portable/streams/PortableOutputStream.java  |  259 --
 .../portable/streams/PortableStream.java        |   53 -
 .../internal/portable/streams/package-info.java |   22 -
 .../processors/GridProcessorAdapter.java        |    5 +-
 .../CacheDefaultBinaryAffinityKeyMapper.java    |   10 +-
 .../processors/cache/CacheInvokeEntry.java      |   10 +-
 .../processors/cache/CacheLazyEntry.java        |   34 +-
 .../processors/cache/CacheObjectContext.java    |   74 +-
 .../processors/cache/CacheOperationContext.java |   12 +-
 .../processors/cache/GridCacheAdapter.java      |  168 +-
 .../cache/GridCacheConcurrentMap.java           |  736 +---
 .../processors/cache/GridCacheContext.java      |   44 +-
 .../processors/cache/GridCacheEntryEx.java      |    8 +-
 .../processors/cache/GridCacheEventManager.java |   24 +-
 .../processors/cache/GridCacheIoManager.java    |    5 +-
 .../processors/cache/GridCacheMapEntry.java     |   98 +-
 .../cache/GridCacheMapEntryFactory.java         |   13 +-
 .../cache/GridCacheOffheapSwapEntry.java        |    4 +-
 .../processors/cache/GridCacheProcessor.java    |   45 +-
 .../processors/cache/GridCacheProxyImpl.java    |    2 +-
 .../processors/cache/GridCacheReturn.java       |   10 +-
 .../processors/cache/IgniteCacheProxy.java      |   36 +-
 .../processors/cache/IgniteInternalCache.java   |   48 +-
 .../cache/binary/BinaryMetadataKey.java         |   82 +
 .../CacheDefaultBinaryAffinityKeyMapper.java    |   51 +
 .../cache/binary/CacheObjectBinaryContext.java  |   56 +
 .../binary/CacheObjectBinaryProcessor.java      |  106 +
 .../binary/CacheObjectBinaryProcessorImpl.java  |  911 ++++
 .../cache/binary/IgniteBinaryImpl.java          |  178 +
 .../processors/cache/binary/package-info.java   |   22 +
 .../CacheDataStructuresManager.java             |   11 +-
 .../distributed/GridDistributedCacheEntry.java  |   14 +-
 .../distributed/GridDistributedLockRequest.java |    2 +-
 .../dht/CacheDistributedGetFutureAdapter.java   |   10 +-
 .../dht/GridDhtAffinityAssignmentResponse.java  |    2 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   21 +-
 .../distributed/dht/GridDhtCacheEntry.java      |   14 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |    4 +-
 .../dht/GridDhtOffHeapCacheEntry.java           |   15 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    2 +-
 .../distributed/dht/GridNoStorageCacheMap.java  |    4 +-
 .../dht/GridPartitionedGetFuture.java           |   14 +-
 .../dht/GridPartitionedSingleGetFuture.java     |   26 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   49 +-
 .../dht/atomic/GridDhtAtomicCacheEntry.java     |   16 +-
 .../atomic/GridDhtAtomicOffHeapCacheEntry.java  |   15 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |    4 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |    4 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |    4 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   49 +-
 .../colocated/GridDhtColocatedCacheEntry.java   |   14 +-
 .../GridDhtColocatedOffHeapCacheEntry.java      |   15 +-
 .../colocated/GridDhtDetachedCacheEntry.java    |    4 +-
 .../GridDhtPartitionsExchangeFuture.java        |    4 +-
 .../preloader/GridDhtPartitionsFullMessage.java |    8 +-
 .../distributed/near/GridNearAtomicCache.java   |   10 +-
 .../distributed/near/GridNearCacheAdapter.java  |   18 +-
 .../distributed/near/GridNearCacheEntry.java    |   22 +-
 .../distributed/near/GridNearGetFuture.java     |   22 +-
 .../near/GridNearOffHeapCacheEntry.java         |   15 +-
 .../near/GridNearTransactionalCache.java        |   12 +-
 .../cache/distributed/near/GridNearTxLocal.java |    6 +-
 .../cache/jta/CacheJtaManagerAdapter.java       |    4 +-
 .../processors/cache/local/GridLocalCache.java  |    8 +-
 .../cache/local/GridLocalCacheEntry.java        |   15 +-
 .../local/atomic/GridLocalAtomicCache.java      |   82 +-
 .../CacheDefaultPortableAffinityKeyMapper.java  |   51 -
 .../portable/CacheObjectBinaryProcessor.java    |  106 -
 .../CacheObjectBinaryProcessorImpl.java         |  904 ----
 .../portable/CacheObjectPortableContext.java    |   56 -
 .../cache/portable/IgniteBinaryImpl.java        |  178 -
 .../cache/portable/PortableMetadataKey.java     |   82 -
 .../processors/cache/portable/package-info.java |   22 -
 .../query/GridCacheDistributedQueryManager.java |    8 +-
 .../cache/query/GridCacheQueryAdapter.java      |   32 +-
 .../query/GridCacheQueryFutureAdapter.java      |    4 +-
 .../cache/query/GridCacheQueryManager.java      |   64 +-
 .../cache/query/GridCacheQueryRequest.java      |   22 +-
 .../continuous/CacheContinuousQueryEvent.java   |    8 +-
 .../cache/store/CacheOsStoreManager.java        |   12 +-
 .../cache/store/CacheStoreManager.java          |   10 +-
 .../store/GridCacheStoreManagerAdapter.java     |   18 +-
 .../transactions/IgniteTxLocalAdapter.java      |   44 +-
 .../cache/transactions/IgniteTxLocalEx.java     |    6 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |   16 +-
 .../IgniteCacheObjectProcessorImpl.java         |    6 +-
 .../processors/cacheobject/NoOpBinary.java      |    2 +-
 .../datastreamer/DataStreamerEntry.java         |    6 +-
 .../datastructures/DataStructuresProcessor.java |  165 +-
 .../GridTransactionalCacheQueueImpl.java        |   32 +-
 .../jobmetrics/GridJobMetricsProcessor.java     |    8 +-
 .../platform/PlatformAbstractPredicate.java     |    6 +-
 .../platform/PlatformAbstractTarget.java        |   20 +-
 .../processors/platform/PlatformContext.java    |   10 +-
 .../platform/PlatformContextImpl.java           |   27 +-
 .../platform/PlatformExtendedException.java     |    4 +-
 .../platform/PlatformNoopProcessor.java         |    4 +-
 .../processors/platform/PlatformProcessor.java  |    8 +-
 .../platform/PlatformProcessorImpl.java         |   38 +-
 .../platform/cache/PlatformCache.java           |   34 +-
 .../cache/PlatformCacheEntryFilterImpl.java     |    6 +-
 .../cache/PlatformCacheEntryProcessorImpl.java  |   16 +-
 .../platform/cache/PlatformCacheIterator.java   |    4 +-
 .../PlatformCachePartialUpdateException.java    |   16 +-
 .../cache/affinity/PlatformAffinity.java        |    6 +-
 .../query/PlatformAbstractQueryCursor.java      |    4 +-
 .../PlatformContinuousQueryRemoteFilter.java    |    4 +-
 .../cache/query/PlatformFieldsQueryCursor.java  |    4 +-
 .../cache/query/PlatformQueryCursor.java        |    4 +-
 .../cache/store/PlatformCacheStoreCallback.java |    4 +-
 .../callback/PlatformCallbackGateway.java       |    8 +-
 .../callback/PlatformCallbackUtils.java         |    6 +-
 .../platform/cluster/PlatformClusterGroup.java  |    6 +-
 .../cluster/PlatformClusterNodeFilterImpl.java  |    6 +-
 .../platform/compute/PlatformAbstractJob.java   |    4 +-
 .../platform/compute/PlatformAbstractTask.java  |    4 +-
 .../platform/compute/PlatformClosureJob.java    |    4 +-
 .../platform/compute/PlatformCompute.java       |   20 +-
 .../platform/compute/PlatformFullJob.java       |    4 +-
 .../platform/compute/PlatformFullTask.java      |    6 +-
 .../cpp/PlatformCppConfigurationClosure.java    |    4 +-
 .../datastreamer/PlatformDataStreamer.java      |   15 +-
 .../PlatformStreamReceiverImpl.java             |   20 +-
 .../dotnet/PlatformDotNetCacheStore.java        |   18 +-
 .../PlatformDotNetConfigurationClosure.java     |   20 +-
 .../dotnet/PlatformDotNetServiceImpl.java       |    8 +-
 .../events/PlatformEventFilterListenerImpl.java |    6 +-
 .../platform/events/PlatformEvents.java         |    4 +-
 .../platform/memory/PlatformInputStream.java    |    6 +-
 .../platform/memory/PlatformOutputStream.java   |    6 +-
 .../messaging/PlatformMessageFilterImpl.java    |    6 +-
 .../messaging/PlatformMessageLocalFilter.java   |    2 +-
 .../platform/messaging/PlatformMessaging.java   |    6 +-
 .../services/PlatformAbstractService.java       |   32 +-
 .../platform/services/PlatformService.java      |    6 +-
 .../platform/services/PlatformServices.java     |   32 +-
 .../transactions/PlatformTransactions.java      |    4 +-
 .../platform/utils/PlatformFutureUtils.java     |    4 +-
 .../platform/utils/PlatformReaderBiClosure.java |    4 +-
 .../platform/utils/PlatformReaderClosure.java   |    4 +-
 .../platform/utils/PlatformUtils.java           |    6 +-
 .../platform/utils/PlatformWriterBiClosure.java |    4 +-
 .../platform/utils/PlatformWriterClosure.java   |    4 +-
 .../query/GridQueryCacheObjectsIterator.java    |   14 +-
 .../processors/query/GridQueryProcessor.java    |   91 +-
 .../message/GridClientBinaryMetaData.java       |   71 +
 .../message/GridClientPortableMetaData.java     |   71 -
 .../client/message/GridClientTaskRequest.java   |   14 +-
 .../internal/util/GridMessageCollection.java    |  160 +
 .../ignite/internal/util/IgniteUtils.java       |   18 +-
 .../internal/util/UUIDCollectionMessage.java    |   42 +-
 .../util/nio/GridNioBackPressureControl.java    |   14 +-
 .../ignite/internal/util/nio/GridNioServer.java |   11 +-
 .../ignite/internal/visor/cache/VisorCache.java |   13 +
 .../internal/visor/query/VisorQueryJob.java     |   76 +-
 .../internal/visor/query/VisorQueryUtils.java   |    5 +-
 .../internal/visor/util/VisorMimeTypes.java     |   10 +-
 .../marshaller/optimized/package-info.java      |    2 +-
 .../marshaller/portable/package-info.java       |   22 -
 .../PlatformDotNetBinaryConfiguration.java      |    6 +-
 .../dotnet/PlatformDotNetConfiguration.java     |    8 +-
 .../org/apache/ignite/spi/IgniteSpiThread.java  |   10 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |    4 +-
 .../communication/tcp/TcpCommunicationSpi.java  |    6 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   17 +-
 .../TcpDiscoveryMulticastIpFinder.java          |  106 +-
 .../org/apache/ignite/thread/IgniteThread.java  |   45 +-
 .../ignite/thread/IgniteThreadPoolExecutor.java |   37 +-
 .../java/org/jsr166/ConcurrentHashMap8.java     |    2 +-
 .../resources/META-INF/classnames.properties    |   68 +-
 .../core/src/main/resources/ignite.properties   |    2 +-
 .../ignite/binary/test1/1.1/test1-1.1.jar       |  Bin 0 -> 2548 bytes
 .../ignite/binary/test1/1.1/test1-1.1.pom       |    9 +
 .../binary/test1/maven-metadata-local.xml       |   12 +
 .../ignite/binary/test2/1.1/test2-1.1.jar       |  Bin 0 -> 1361 bytes
 .../ignite/binary/test2/1.1/test2-1.1.pom       |    9 +
 .../binary/test2/maven-metadata-local.xml       |   12 +
 ...heJdbcPojoStoreBinaryMarshallerSelfTest.java |    2 +-
 ...eJdbcStoreAbstractMultithreadedSelfTest.java |    5 +-
 ...CacheJdbcBlobStoreMultithreadedSelfTest.java |    2 +-
 .../GridEventStorageCheckAllEventsSelfTest.java |   22 +-
 .../IgniteClientReconnectAbstractTest.java      |   96 +-
 .../IgniteClientReconnectAtomicsTest.java       |   57 +
 .../IgniteClientReconnectCacheTest.java         |   15 +-
 .../IgniteClientReconnectCollectionsTest.java   |   51 +
 .../internal/binary/BinaryEnumsSelfTest.java    |  446 ++
 .../binary/BinaryFieldsAbstractSelfTest.java    |  718 +++
 .../binary/BinaryFieldsHeapSelfTest.java        |   30 +
 .../binary/BinaryFieldsOffheapSelfTest.java     |   60 +
 .../BinaryFooterOffsetsAbstractSelfTest.java    |  205 +
 .../binary/BinaryFooterOffsetsHeapSelfTest.java |   30 +
 .../BinaryFooterOffsetsOffheapSelfTest.java     |   60 +
 .../binary/BinaryMarshallerSelfTest.java        | 4167 ++++++++++++++++++
 .../BinaryObjectBuilderAdditionalSelfTest.java  | 1264 ++++++
 .../binary/BinaryObjectBuilderSelfTest.java     | 1108 +++++
 .../binary/GridBinaryAffinityKeySelfTest.java   |  234 +
 ...GridBinaryMarshallerCtxDisabledSelfTest.java |  247 ++
 .../binary/GridBinaryMetaDataSelfTest.java      |  371 ++
 .../binary/GridBinaryWildcardsSelfTest.java     |  464 ++
 .../binary/TestCachingMetadataHandler.java      |   45 +
 .../GridBinaryMarshalerAwareTestClass.java      |   67 +
 .../mutabletest/GridBinaryTestClasses.java      |  484 ++
 .../binary/mutabletest/package-info.java        |   22 +
 .../BinaryFieldsHeapNonCompactSelfTest.java     |   30 +
 .../BinaryFieldsOffheapNonCompactSelfTest.java  |   30 +
 ...naryFooterOffsetsHeapNonCompactSelfTest.java |   30 +
 ...yFooterOffsetsOffheapNonCompactSelfTest.java |   30 +
 .../BinaryMarshallerNonCompactSelfTest.java     |   30 +
 ...jectBuilderAdditionalNonCompactSelfTest.java |   30 +
 .../BinaryObjectBuilderNonCompactSelfTest.java  |   30 +
 .../ignite/internal/binary/package-info.java    |   22 +
 .../binary/test/GridBinaryTestClass1.java       |   28 +
 .../binary/test/GridBinaryTestClass2.java       |   24 +
 .../internal/binary/test/package-info.java      |   22 +
 .../test/subpackage/GridBinaryTestClass3.java   |   24 +
 .../binary/test/subpackage/package-info.java    |   22 +
 .../internal/portable/BinaryEnumsSelfTest.java  |  446 --
 .../portable/BinaryFieldsAbstractSelfTest.java  |  718 ---
 .../portable/BinaryFieldsHeapSelfTest.java      |   30 -
 .../portable/BinaryFieldsOffheapSelfTest.java   |   60 -
 .../BinaryFooterOffsetsAbstractSelfTest.java    |  205 -
 .../BinaryFooterOffsetsHeapSelfTest.java        |   30 -
 .../BinaryFooterOffsetsOffheapSelfTest.java     |   60 -
 .../portable/BinaryMarshallerSelfTest.java      | 3877 ----------------
 .../BinaryObjectBuilderAdditionalSelfTest.java  | 1291 ------
 .../portable/BinaryObjectBuilderSelfTest.java   | 1103 -----
 .../GridPortableAffinityKeySelfTest.java        |  234 -
 ...idPortableMarshallerCtxDisabledSelfTest.java |  247 --
 .../portable/GridPortableMetaDataSelfTest.java  |  371 --
 .../portable/GridPortableWildcardsSelfTest.java |  464 --
 .../portable/TestCachingMetadataHandler.java    |   45 -
 .../GridBinaryMarshalerAwareTestClass.java      |   67 -
 .../mutabletest/GridPortableTestClasses.java    |  443 --
 .../portable/mutabletest/package-info.java      |   22 -
 .../BinaryFieldsHeapNonCompactSelfTest.java     |   30 -
 .../BinaryFieldsOffheapNonCompactSelfTest.java  |   30 -
 ...naryFooterOffsetsHeapNonCompactSelfTest.java |   30 -
 ...yFooterOffsetsOffheapNonCompactSelfTest.java |   30 -
 .../BinaryMarshallerNonCompactSelfTest.java     |   30 -
 ...jectBuilderAdditionalNonCompactSelfTest.java |   30 -
 .../BinaryObjectBuilderNonCompactSelfTest.java  |   30 -
 .../ignite/internal/portable/package-info.java  |   22 -
 .../portable/test/GridPortableTestClass1.java   |   28 -
 .../portable/test/GridPortableTestClass2.java   |   24 -
 .../internal/portable/test/package-info.java    |   22 -
 .../test/subpackage/GridPortableTestClass3.java |   24 -
 .../portable/test/subpackage/package-info.java  |   22 -
 ...cheAbstractFullApiMultithreadedSelfTest.java |    2 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |   29 +-
 .../cache/GridCacheAlwaysEvictionPolicy.java    |    2 +-
 ...eAtomicEntryProcessorDeploymentSelfTest.java |   69 +-
 .../cache/GridCacheConcurrentMapSelfTest.java   |   11 -
 .../GridCacheConditionalDeploymentSelfTest.java |   20 +-
 .../cache/GridCacheDeploymentSelfTest.java      |   22 +-
 .../GridCacheOffHeapTieredAbstractSelfTest.java |    8 +-
 .../GridCacheOnCopyFlagAbstractSelfTest.java    |    6 +-
 .../processors/cache/GridCacheTestEntryEx.java  |    8 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |    8 +-
 .../IgniteCacheStoreValueAbstractTest.java      |    6 +-
 .../IgniteStartCacheInTransactionSelfTest.java  |    8 +
 ...gniteTopologyValidatorAbstractCacheTest.java |   18 +-
 ...iteTopologyValidatorAbstractTxCacheTest.java |    1 -
 .../IgniteTxExceptionAbstractSelfTest.java      |   59 +-
 .../GridBinaryCacheEntryMemorySizeSelfTest.java |   48 +
 ...ryDuplicateIndexObjectsAbstractSelfTest.java |  161 +
 ...yAtomicEntryProcessorDeploymentSelfTest.java |  129 +
 ...naryObjectsAbstractDataStreamerSelfTest.java |  192 +
 ...aryObjectsAbstractMultiThreadedSelfTest.java |  241 +
 .../GridCacheBinaryObjectsAbstractSelfTest.java |  981 +++++
 .../GridCacheBinaryStoreAbstractSelfTest.java   |  300 ++
 .../GridCacheBinaryStoreBinariesSelfTest.java   |   66 +
 .../GridCacheBinaryStoreObjectsSelfTest.java    |   55 +
 ...ctionalEntryProcessorDeploymentSelfTest.java |   31 +
 ...ntNodeBinaryObjectMetadataMultinodeTest.java |  295 ++
 ...CacheClientNodeBinaryObjectMetadataTest.java |  221 +
 .../DataStreamProcessorBinarySelfTest.java      |   71 +
 .../GridDataStreamerImplSelfTest.java           |  345 ++
 .../GridCacheAffinityRoutingBinarySelfTest.java |   54 +
 ...OnlyBinaryDataStreamerMultiNodeSelfTest.java |   29 +
 ...BinaryDataStreamerMultithreadedSelfTest.java |   47 +
 ...cPartitionedOnlyBinaryMultiNodeSelfTest.java |   28 +
 ...titionedOnlyBinaryMultithreadedSelfTest.java |   47 +
 ...sNearPartitionedByteArrayValuesSelfTest.java |   41 +
 ...sPartitionedOnlyByteArrayValuesSelfTest.java |   42 +
 ...ateIndexObjectPartitionedAtomicSelfTest.java |   38 +
 ...xObjectPartitionedTransactionalSelfTest.java |   41 +
 ...AtomicNearDisabledOffheapTieredSelfTest.java |   29 +
 ...BinaryObjectsAtomicNearDisabledSelfTest.java |   51 +
 ...inaryObjectsAtomicOffheapTieredSelfTest.java |   29 +
 .../GridCacheBinaryObjectsAtomicSelfTest.java   |   51 +
 ...tionedNearDisabledOffheapTieredSelfTest.java |   30 +
 ...yObjectsPartitionedNearDisabledSelfTest.java |   51 +
 ...ObjectsPartitionedOffheapTieredSelfTest.java |   30 +
 ...idCacheBinaryObjectsPartitionedSelfTest.java |   51 +
 .../dht/GridCacheMemoryModeBinarySelfTest.java  |   36 +
 ...dCacheOffHeapTieredAtomicBinarySelfTest.java |   48 +
 .../GridCacheOffHeapTieredBinarySelfTest.java   |   48 +
 ...fHeapTieredEvictionAtomicBinarySelfTest.java |   96 +
 ...acheOffHeapTieredEvictionBinarySelfTest.java |   96 +
 ...ridCacheBinaryObjectsReplicatedSelfTest.java |   51 +
 ...idCacheBinaryObjectsAtomicLocalSelfTest.java |   32 +
 ...BinaryObjectsLocalOffheapTieredSelfTest.java |   29 +
 .../GridCacheBinaryObjectsLocalSelfTest.java    |   51 +
 ...IgniteCacheAbstractExecutionContextTest.java |    7 +-
 ...eAbstractDataStructuresFailoverSelfTest.java |  140 +-
 .../CacheTryLockMultithreadedTest.java          |  116 +
 .../GridCacheAbstractNodeRestartSelfTest.java   |   25 +-
 ...niteCacheClientNodeChangingTopologyTest.java |  125 +-
 .../near/GridCacheNearTxExceptionSelfTest.java  |   20 -
 .../near/NearCachePutAllMultinodeTest.java      |  167 +
 ...eRebalancingUnmarshallingFailedSelfTest.java |    2 +-
 ...idCacheReplicatedUnswapAdvancedSelfTest.java |   44 +-
 .../GridCacheReplicatedPreloadSelfTest.java     |   44 +-
 .../BinaryAtomicCacheLocalEntriesSelfTest.java  |   30 +
 .../BinaryTxCacheLocalEntriesSelfTest.java      |   84 +
 ...naryObjectsAbstractDataStreamerSelfTest.java |  192 -
 ...aryObjectsAbstractMultiThreadedSelfTest.java |  241 -
 .../GridCacheBinaryObjectsAbstractSelfTest.java |  981 -----
 ...ntNodeBinaryObjectMetadataMultinodeTest.java |  295 --
 ...CacheClientNodeBinaryObjectMetadataTest.java |  221 -
 .../GridCachePortableStoreAbstractSelfTest.java |  300 --
 .../GridCachePortableStoreObjectsSelfTest.java  |   55 -
 ...GridCachePortableStorePortablesSelfTest.java |   66 -
 ...ridPortableCacheEntryMemorySizeSelfTest.java |   48 -
 ...leDuplicateIndexObjectsAbstractSelfTest.java |  161 -
 .../DataStreamProcessorPortableSelfTest.java    |   71 -
 .../GridDataStreamerImplSelfTest.java           |  345 --
 ...ridCacheAffinityRoutingPortableSelfTest.java |   54 -
 ...lyPortableDataStreamerMultiNodeSelfTest.java |   29 -
 ...rtableDataStreamerMultithreadedSelfTest.java |   47 -
 ...artitionedOnlyPortableMultiNodeSelfTest.java |   28 -
 ...tionedOnlyPortableMultithreadedSelfTest.java |   47 -
 ...AtomicNearDisabledOffheapTieredSelfTest.java |   29 -
 ...BinaryObjectsAtomicNearDisabledSelfTest.java |   51 -
 ...inaryObjectsAtomicOffheapTieredSelfTest.java |   29 -
 .../GridCacheBinaryObjectsAtomicSelfTest.java   |   51 -
 ...tionedNearDisabledOffheapTieredSelfTest.java |   30 -
 ...yObjectsPartitionedNearDisabledSelfTest.java |   51 -
 ...ObjectsPartitionedOffheapTieredSelfTest.java |   30 -
 ...idCacheBinaryObjectsPartitionedSelfTest.java |   51 -
 .../GridCacheMemoryModePortableSelfTest.java    |   36 -
 ...acheOffHeapTieredAtomicPortableSelfTest.java |   48 -
 ...eapTieredEvictionAtomicPortableSelfTest.java |   96 -
 ...heOffHeapTieredEvictionPortableSelfTest.java |   96 -
 .../GridCacheOffHeapTieredPortableSelfTest.java |   48 -
 ...ateIndexObjectPartitionedAtomicSelfTest.java |   38 -
 ...xObjectPartitionedTransactionalSelfTest.java |   41 -
 ...sNearPartitionedByteArrayValuesSelfTest.java |   41 -
 ...sPartitionedOnlyByteArrayValuesSelfTest.java |   42 -
 ...ridCacheBinaryObjectsReplicatedSelfTest.java |   51 -
 ...idCacheBinaryObjectsAtomicLocalSelfTest.java |   32 -
 ...BinaryObjectsLocalOffheapTieredSelfTest.java |   29 -
 .../GridCacheBinaryObjectsLocalSelfTest.java    |   51 -
 ...ContinuousQueryFailoverAbstractSelfTest.java |  186 +-
 .../DataStreamProcessorSelfTest.java            |   25 +-
 .../loadtests/hashmap/GridHashMapLoadTest.java  |    4 +-
 .../marshaller/GridMarshallerAbstractTest.java  |   32 +-
 .../PlatformComputeBinarizableArgTask.java      |    4 +-
 .../platform/PlatformComputeEchoTask.java       |   20 +-
 .../platform/PlatformEventsWriteEventTask.java  |    4 +-
 .../GridSessionCheckpointAbstractSelfTest.java  |   32 +-
 .../session/GridSessionCheckpointSelfTest.java  |   12 +-
 ...lientDiscoverySpiFailureTimeoutSelfTest.java |    5 +-
 .../tcp/TcpClientDiscoverySpiMulticastTest.java |   97 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |  130 +-
 .../TcpDiscoveryIpFinderAbstractSelfTest.java   |    2 +-
 .../TcpDiscoveryMulticastIpFinderSelfTest.java  |   16 +-
 .../testframework/junits/GridAbstractTest.java  |   11 +-
 .../ignite/testframework/junits/IgniteMock.java |   10 +-
 .../junits/IgniteTestResources.java             |   14 +-
 .../IgniteBinaryCacheFullApiTestSuite.java      |   37 +
 .../testsuites/IgniteBinaryCacheTestSuite.java  |  101 +
 ...ObjectsCacheDataStructuresSelfTestSuite.java |    2 +-
 ...BinaryObjectsCacheExpiryPolicyTestSuite.java |    2 +-
 ...gniteBinaryObjectsCacheRestartTestSuite.java |    2 +-
 .../IgniteBinaryObjectsCacheTestSuite2.java     |    2 +-
 .../IgniteBinaryObjectsCacheTestSuite3.java     |   13 +-
 .../IgniteBinaryObjectsCacheTestSuite4.java     |    2 +-
 ...IgniteBinaryObjectsComputeGridTestSuite.java |    2 +-
 .../IgniteBinaryObjectsTestSuite.java           |  119 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |    4 +-
 .../IgnitePortableCacheFullApiTestSuite.java    |   37 -
 .../IgnitePortableCacheTestSuite.java           |  102 -
 .../IgnitePortableObjectsTestSuite.java         |  114 -
 .../testsuites/IgniteUtilSelfTestSuite.java     |    2 +
 .../ignite/util/GridMessageCollectionTest.java  |  127 +
 .../ignite/portable/test1/1.1/test1-1.1.jar     |  Bin 2548 -> 0 bytes
 .../ignite/portable/test1/1.1/test1-1.1.pom     |    9 -
 .../portable/test1/maven-metadata-local.xml     |   12 -
 .../ignite/portable/test2/1.1/test2-1.1.jar     |  Bin 1361 -> 0 bytes
 .../ignite/portable/test2/1.1/test2-1.1.pom     |    9 -
 .../portable/test2/maven-metadata-local.xml     |   12 -
 modules/docker/1.0.0/Dockerfile                 |   40 +
 modules/docker/1.0.0/run.sh                     |   50 +
 modules/docker/1.1.0/Dockerfile                 |   40 +
 modules/docker/1.1.0/run.sh                     |   50 +
 modules/docker/1.2.0/Dockerfile                 |   40 +
 modules/docker/1.2.0/run.sh                     |   50 +
 modules/docker/1.3.0/Dockerfile                 |   40 +
 modules/docker/1.3.0/run.sh                     |   50 +
 modules/docker/1.4.0/Dockerfile                 |   40 +
 modules/docker/1.4.0/run.sh                     |   50 +
 modules/docker/Dockerfile                       |   41 +-
 modules/docker/build_users_libs.sh              |   39 -
 modules/docker/download_ignite.sh               |   49 -
 modules/docker/execute.sh                       |   62 -
 modules/docker/run.sh                           |   36 +-
 modules/extdata/p2p/pom.xml                     |    2 +-
 .../CacheDeploymentBinaryEntryProcessor.java    |   35 +
 ...cheDeploymentBinaryObjectEntryProcessor.java |   36 +
 .../CacheDeploymentPortableEntryProcessor.java  |   35 -
 .../tests/p2p/CacheDeploymentTestValue3.java    |   41 +
 .../apache/ignite/tests/p2p/cache/Person.java   |   77 +
 .../ignite/tests/p2p/cache/PersonKey.java       |   74 +
 .../extdata/uri/modules/uri-dependency/pom.xml  |    2 +-
 modules/extdata/uri/pom.xml                     |    2 +-
 modules/flume/pom.xml                           |    2 +-
 modules/gce/pom.xml                             |    2 +-
 modules/geospatial/pom.xml                      |    2 +-
 modules/hadoop/pom.xml                          |    2 +-
 modules/hibernate/pom.xml                       |    2 +-
 .../HibernateTransactionalDataRegion.java       |   18 +-
 .../HibernateL2CacheTransactionalSelfTest.java  |   23 +-
 modules/indexing/pom.xml                        |    2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   13 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |    2 +-
 .../IgniteBinaryObjectFieldsQuerySelfTest.java  |  246 ++
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |   15 +-
 .../query/IgniteSqlSplitterSelfTest.java        |    4 +-
 .../IgniteBinaryCacheQueryTestSuite.java        |  116 +
 .../IgnitePortableCacheQueryTestSuite.java      |  114 -
 modules/jcl/pom.xml                             |    2 +-
 modules/jms11/pom.xml                           |    2 +-
 modules/jta/pom.xml                             |   11 +-
 .../cache/jta/jndi/CacheJndiTmFactory.java      |  132 +
 .../processors/cache/jta/CacheJtaManager.java   |   41 +-
 .../cache/AbstarctCacheJtaSelfTest.java         |  183 +
 .../cache/CacheJndiTmFactorySelfTest.java       |  166 +
 ...CacheJtaConfigurationValidationSelfTest.java |    4 +-
 ...CacheJtaFactoryConfigValidationSelfTest.java |  142 +
 .../processors/cache/GridCacheJtaSelfTest.java  |  221 -
 .../cache/GridCacheReplicatedJtaSelfTest.java   |   32 -
 .../cache/GridJtaLifecycleAwareSelfTest.java    |  191 +
 .../GridPartitionedCacheJtaFactorySelfTest.java |   41 +
 ...titionedCacheJtaLookupClassNameSelfTest.java |   83 +
 .../GridReplicatedCacheJtaFactorySelfTest.java  |   32 +
 ...plicatedCacheJtaLookupClassNameSelfTest.java |   32 +
 .../GridTmLookupLifecycleAwareSelfTest.java     |  122 -
 .../ignite/testsuites/IgniteJtaTestSuite.java   |   26 +-
 modules/kafka/pom.xml                           |    2 +-
 modules/log4j/pom.xml                           |    2 +-
 modules/log4j2/pom.xml                          |    2 +-
 modules/mesos/pom.xml                           |    2 +-
 modules/mqtt/pom.xml                            |    2 +-
 modules/osgi-karaf/pom.xml                      |    5 +-
 .../osgi-karaf/src/main/resources/features.xml  |   11 +-
 modules/osgi-paxlogging/pom.xml                 |    6 +-
 modules/osgi/pom.xml                            |    5 +-
 .../ignite/osgi/classloaders/package-info.java  |   21 +
 .../org/apache/ignite/osgi/package-info.java    |   21 +
 .../ignite/osgi/AbstractIgniteKarafTest.java    |    2 -
 .../IgniteKarafFeaturesInstallationTest.java    |    2 +-
 .../ignite/osgi/IgniteOsgiServiceTest.java      |    2 +-
 modules/platforms/cpp/common/configure.ac       |    2 +-
 modules/platforms/cpp/common/src/java.cpp       |    8 +-
 modules/platforms/cpp/core-test/configure.ac    |    2 +-
 .../src/binary_reader_writer_raw_test.cpp       |   10 +-
 .../core-test/src/binary_reader_writer_test.cpp |   10 +-
 modules/platforms/cpp/core/configure.ac         |    2 +-
 .../core/include/ignite/binary/binary_consts.h  |   29 +-
 .../cpp/core/include/ignite/ignition.h          |   41 +-
 modules/platforms/cpp/cpp.dxg                   | 1722 ++++++++
 modules/platforms/cpp/examples/configure.ac     |    2 +-
 modules/platforms/cpp/ignite/configure.ac       |    2 +-
 .../Properties/AssemblyInfo.cs                  |    4 +-
 .../Properties/AssemblyInfo.cs                  |    4 +-
 .../Binary/BinarySelfTest.cs                    |   67 +
 .../Config/marshaller-explicit.xml              |    2 +-
 .../Properties/AssemblyInfo.cs                  |    4 +-
 .../Apache.Ignite.Core.csproj                   |    5 +
 .../Cache/Query/IQueryCursor.cs                 |    2 +-
 .../Apache.Ignite.Core/Cache/Query/QueryBase.cs |    2 +-
 .../Store/CacheParallelLoadStoreAdapter.cs      |    4 +-
 .../Cache/Store/CacheStoreAdapter.cs            |   12 +-
 .../Cache/Store/ICacheStore.cs                  |   36 +-
 .../Apache.Ignite.Core/Cluster/ICluster.cs      |    2 +-
 .../Cluster/IClusterMetrics.cs                  |    6 +-
 .../Apache.Ignite.Core/Cluster/IClusterNode.cs  |    6 +-
 .../Apache.Ignite.Core/Compute/IComputeTask.cs  |    2 +-
 .../Datastream/IDataStreamer.cs                 |   12 +-
 .../Apache.Ignite.Core/Events/CacheEvent.cs     |    4 +-
 .../Apache.Ignite.Core/Events/TaskEvent.cs      |    2 +-
 .../dotnet/Apache.Ignite.Core/IIgnite.cs        |    6 +-
 .../dotnet/Apache.Ignite.Core/Ignition.cs       |   24 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |    8 +-
 .../Impl/Binary/BinaryObjectHeader.cs           |   10 +-
 .../Impl/Binary/BinaryReflectiveActions.cs      |    4 +-
 .../Impl/Binary/BinarySystemHandlers.cs         |   34 +-
 .../Impl/Binary/BinaryUtils.cs                  |   94 +-
 .../Impl/Binary/Metadata/BinaryTypeHolder.cs    |    1 +
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |    2 -
 .../Impl/Common/DelegateConverter.cs            |   15 +-
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |    8 +
 .../Apache.Ignite.Core/Impl/NativeMethods.cs    |   10 -
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |  358 ++
 .../Impl/Unmanaged/UnmanagedUtils.cs            |  788 +---
 .../Properties/AssemblyInfo.cs                  |    4 +-
 .../Transactions/ITransaction.cs                |    2 +-
 modules/platforms/dotnet/Apache.Ignite.FxCop    |  322 +-
 modules/platforms/dotnet/Apache.Ignite.dxg      | 2387 ++++++++++
 modules/platforms/dotnet/Apache.Ignite.sln      |    1 +
 modules/platforms/dotnet/Apache.Ignite.slnrel   |   43 -
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |    4 +-
 .../platforms/dotnet/Apache.Ignite_x86.slnrel   |   43 -
 modules/platforms/dotnet/README.txt             |   36 +-
 modules/platforms/dotnet/build.bat              |   94 +
 modules/platforms/dotnet/dotnet.dxg             | 1795 ++++++++
 .../Apache.Ignite.Examples.csproj               |    4 +-
 .../Properties/AssemblyInfo.cs                  |    4 +-
 .../Apache.Ignite.ExamplesDll.csproj            |    4 +-
 .../Properties/AssemblyInfo.cs                  |    4 +-
 modules/platforms/dotnet/examples/README.txt    |    8 -
 modules/platforms/dotnet/header.html            |   27 +
 modules/rest-http/pom.xml                       |    2 +-
 modules/scalar-2.10/pom.xml                     |    2 +-
 modules/scalar/pom.xml                          |    2 +-
 modules/schedule/pom.xml                        |    2 +-
 modules/schema-import/pom.xml                   |    2 +-
 .../ignite/schema/generator/CodeGenerator.java  |   30 +-
 modules/slf4j/pom.xml                           |    2 +-
 modules/spark-2.10/pom.xml                      |    2 +-
 modules/spark/pom.xml                           |    2 +-
 modules/spring/pom.xml                          |    2 +-
 .../ignite/internal/GridFactorySelfTest.java    |   45 +
 .../GridSpringBeanSerializationSelfTest.java    |    2 +
 modules/ssh/pom.xml                             |    2 +-
 modules/tools/pom.xml                           |    2 +-
 modules/twitter/pom.xml                         |    2 +-
 modules/urideploy/pom.xml                       |    2 +-
 modules/visor-console-2.10/pom.xml              |    2 +-
 modules/visor-console/pom.xml                   |    2 +-
 modules/visor-plugins/pom.xml                   |    2 +-
 modules/web/pom.xml                             |    2 +-
 .../config/benchmark-failover.properties        |    2 +-
 .../ignite-int-max-values-offheap-config.xml    |    2 +-
 .../ignite-int-max-values-onheap-config.xml     |    2 +-
 .../ignite-int-max-values-swap-config.xml       |    2 +-
 modules/yardstick/pom.xml                       |    2 +-
 ...IgniteTransactionalWriteInvokeBenchmark.java |   42 +-
 modules/yarn/pom.xml                            |    2 +-
 modules/zookeeper/pom.xml                       |    2 +-
 parent/pom.xml                                  |   15 +-
 pom.xml                                         |    6 +-
 759 files changed, 48990 insertions(+), 39597 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dddb817b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddb817b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index e387c0a,3c7f378..769f043
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@@ -720,9 -721,14 +721,14 @@@ public class GridIoMessageFactory imple
  
                  break;
  
-             // [-3..119] - this
+             case 124:
+                 msg = new GridMessageCollection<>();
+ 
+                 break;
+ 
+             // [-3..119] [124] - this
              // [120..123] - DR
 -            // [-4..-22] - SQL
 +            // [-4..-28] - SQL
              default:
                  if (ext != null) {
                      for (MessageFactory factory : ext) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddb817b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
index 0000000,e77b85a..15dd088
mode 000000,100644..100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
@@@ -1,0 -1,903 +1,911 @@@
+ /*
+  * 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.binary;
+ 
+ import java.io.Externalizable;
+ import java.io.IOException;
+ import java.io.ObjectInput;
+ import java.io.ObjectOutput;
+ import java.util.ArrayList;
+ import java.util.Collection;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.UUID;
+ import java.util.concurrent.ConcurrentHashMap;
+ import java.util.concurrent.CountDownLatch;
+ import javax.cache.Cache;
+ import javax.cache.CacheException;
+ import javax.cache.event.CacheEntryEvent;
+ import javax.cache.event.CacheEntryListenerException;
+ import javax.cache.event.CacheEntryUpdatedListener;
+ import javax.cache.event.EventType;
+ import javax.cache.processor.EntryProcessor;
+ import javax.cache.processor.MutableEntry;
+ import org.apache.ignite.IgniteBinary;
+ import org.apache.ignite.IgniteCheckedException;
+ import org.apache.ignite.IgniteException;
+ import org.apache.ignite.binary.BinaryObject;
+ import org.apache.ignite.binary.BinaryObjectBuilder;
+ import org.apache.ignite.binary.BinaryObjectException;
+ import org.apache.ignite.binary.BinaryType;
+ import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
+ import org.apache.ignite.cluster.ClusterNode;
+ import org.apache.ignite.cluster.ClusterTopologyException;
+ import org.apache.ignite.configuration.CacheConfiguration;
+ import org.apache.ignite.internal.GridKernalContext;
+ import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+ import org.apache.ignite.internal.binary.BinaryEnumObjectImpl;
+ import org.apache.ignite.internal.binary.BinaryMarshaller;
+ import org.apache.ignite.internal.binary.BinaryMetadata;
+ import org.apache.ignite.internal.binary.BinaryMetadataHandler;
+ import org.apache.ignite.internal.binary.BinaryObjectEx;
+ import org.apache.ignite.internal.binary.BinaryObjectImpl;
+ import org.apache.ignite.internal.binary.BinaryObjectOffheapImpl;
+ import org.apache.ignite.internal.binary.BinaryTypeImpl;
+ import org.apache.ignite.internal.binary.GridBinaryMarshaller;
+ import org.apache.ignite.internal.binary.BinaryContext;
+ import org.apache.ignite.internal.binary.BinaryUtils;
+ import org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl;
+ import org.apache.ignite.internal.binary.streams.BinaryInputStream;
+ import org.apache.ignite.internal.binary.streams.BinaryOffheapInputStream;
+ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+ import org.apache.ignite.internal.processors.cache.CacheEntryPredicate;
+ import org.apache.ignite.internal.processors.cache.CacheEntryPredicateAdapter;
+ import org.apache.ignite.internal.processors.cache.CacheObject;
+ import org.apache.ignite.internal.processors.cache.CacheObjectContext;
+ import org.apache.ignite.internal.processors.cache.CacheObjectImpl;
+ import org.apache.ignite.internal.processors.cache.GridCacheContext;
+ import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
+ import org.apache.ignite.internal.processors.cache.GridCacheUtils;
+ import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
+ import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+ import org.apache.ignite.internal.processors.cache.query.CacheQuery;
+ import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture;
+ import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager;
+ import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessorImpl;
+ import org.apache.ignite.internal.util.GridUnsafe;
+ import org.apache.ignite.internal.util.IgniteUtils;
+ import org.apache.ignite.internal.util.lang.GridMapEntry;
+ import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+ import org.apache.ignite.internal.util.typedef.C1;
+ import org.apache.ignite.internal.util.typedef.F;
+ import org.apache.ignite.internal.util.typedef.T2;
+ import org.apache.ignite.internal.util.typedef.X;
+ 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.IgniteBiPredicate;
+ import org.apache.ignite.lang.IgniteBiTuple;
+ import org.apache.ignite.lang.IgniteClosure;
+ import org.apache.ignite.marshaller.Marshaller;
+ import org.jetbrains.annotations.Nullable;
+ import org.jsr166.ConcurrentHashMap8;
+ import sun.misc.Unsafe;
+ 
+ /**
+  * Binary processor implementation.
+  */
+ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorImpl implements
+     CacheObjectBinaryProcessor {
+     /** */
+     private static final Unsafe UNSAFE = GridUnsafe.unsafe();
+ 
+     /** */
+     private final CountDownLatch startLatch = new CountDownLatch(1);
+ 
+     /** */
+     private final boolean clientNode;
+ 
+     /** */
+     private volatile IgniteCacheProxy<BinaryMetadataKey, BinaryMetadata> metaDataCache;
+ 
+     /** */
+     private final ConcurrentHashMap8<Integer, BinaryTypeImpl> clientMetaDataCache;
+ 
+     /** Predicate to filter binary meta data in utility cache. */
+     private final CacheEntryPredicate metaPred = new CacheEntryPredicateAdapter() {
+         private static final long serialVersionUID = 0L;
+ 
+         @Override public boolean apply(GridCacheEntryEx e) {
+             return e.key().value(e.context().cacheObjectContext(), false) instanceof BinaryMetadataKey;
+         }
+     };
+ 
+     /** */
+     private BinaryContext binaryCtx;
+ 
+     /** */
+     private Marshaller marsh;
+ 
+     /** */
+     private GridBinaryMarshaller binaryMarsh;
+ 
+     /** */
+     @GridToStringExclude
+     private IgniteBinary binaries;
+ 
+     /** Metadata updates collected before metadata cache is initialized. */
+     private final Map<Integer, BinaryMetadata> metaBuf = new ConcurrentHashMap<>();
+ 
+     /** */
+     private UUID metaCacheQryId;
+ 
+     /**
+      * @param ctx Kernal context.
+      */
+     public CacheObjectBinaryProcessorImpl(GridKernalContext ctx) {
+         super(ctx);
+ 
+         marsh = ctx.grid().configuration().getMarshaller();
+ 
+         clientNode = this.ctx.clientNode();
+ 
+         clientMetaDataCache = clientNode ? new ConcurrentHashMap8<Integer, BinaryTypeImpl>() : null;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public void start() throws IgniteCheckedException {
+         if (marsh instanceof BinaryMarshaller) {
+             BinaryMetadataHandler metaHnd = new BinaryMetadataHandler() {
+                 @Override public void addMeta(int typeId, BinaryType newMeta) throws BinaryObjectException {
+                     assert newMeta != null;
+                     assert newMeta instanceof BinaryTypeImpl;
+ 
+                     BinaryMetadata newMeta0 = ((BinaryTypeImpl)newMeta).metadata();
+ 
+                     if (metaDataCache == null) {
+                         BinaryMetadata oldMeta = metaBuf.get(typeId);
+                         BinaryMetadata mergedMeta = BinaryUtils.mergeMetadata(oldMeta, newMeta0);
+ 
+                         if (oldMeta != mergedMeta) {
+                             synchronized (this) {
+                                 mergedMeta = BinaryUtils.mergeMetadata(oldMeta, newMeta0);
+ 
+                                 if (oldMeta != mergedMeta)
+                                     metaBuf.put(typeId, mergedMeta);
+                                 else
+                                     return;
+                             }
+ 
+                             if (metaDataCache == null)
+                                 return;
+                             else
+                                 metaBuf.remove(typeId);
+                         }
+                         else
+                             return;
+                     }
+ 
+                     assert metaDataCache != null;
+ 
+                     CacheObjectBinaryProcessorImpl.this.addMeta(typeId, newMeta0.wrap(binaryCtx));
+                 }
+ 
+                 @Override public BinaryType metadata(int typeId) throws BinaryObjectException {
+                     if (metaDataCache == null)
+                         U.awaitQuiet(startLatch);
+ 
+                     return CacheObjectBinaryProcessorImpl.this.metadata(typeId);
+                 }
+             };
+ 
+             BinaryMarshaller pMarh0 = (BinaryMarshaller)marsh;
+ 
+             binaryCtx = new BinaryContext(metaHnd, ctx.config());
+ 
+             IgniteUtils.invoke(BinaryMarshaller.class, pMarh0, "setBinaryContext", binaryCtx,
+                 ctx.config());
+ 
+             binaryMarsh = new GridBinaryMarshaller(binaryCtx);
+ 
+             binaries = new IgniteBinaryImpl(ctx, this);
+         }
+     }
+ 
+     /** {@inheritDoc} */
+     @SuppressWarnings("unchecked")
+     @Override public void onUtilityCacheStarted() throws IgniteCheckedException {
+         IgniteCacheProxy<Object, Object> proxy = ctx.cache().jcache(CU.UTILITY_CACHE_NAME);
+ 
+         boolean old = proxy.context().deploy().ignoreOwnership(true);
+ 
+         try {
+             metaDataCache = (IgniteCacheProxy)proxy.withNoRetries();
+         }
+         finally {
+             proxy.context().deploy().ignoreOwnership(old);
+         }
+ 
+         if (clientNode) {
+             assert !metaDataCache.context().affinityNode();
+ 
+             metaCacheQryId = metaDataCache.context().continuousQueries().executeInternalQuery(
+                 new MetaDataEntryListener(),
+                 new MetaDataEntryFilter(),
+                 false,
+                 true);
+ 
+             while (true) {
+                 ClusterNode oldestSrvNode =
+                     CU.oldestAliveCacheServerNode(ctx.cache().context(), AffinityTopologyVersion.NONE);
+ 
+                 if (oldestSrvNode == null)
+                     break;
+ 
+                 GridCacheQueryManager qryMgr = metaDataCache.context().queries();
+ 
+                 CacheQuery<Map.Entry<BinaryMetadataKey, BinaryMetadata>> qry =
+                     qryMgr.createScanQuery(new MetaDataPredicate(), null, false);
+ 
+                 qry.keepAll(false);
+ 
+                 qry.projection(ctx.cluster().get().forNode(oldestSrvNode));
+ 
+                 try {
+                     CacheQueryFuture<Map.Entry<BinaryMetadataKey, BinaryMetadata>> fut = qry.execute();
+ 
+                     Map.Entry<BinaryMetadataKey, BinaryMetadata> next;
+ 
+                     while ((next = fut.next()) != null) {
+                         assert next.getKey() != null : next;
+                         assert next.getValue() != null : next;
+ 
+                         addClientCacheMetaData(next.getKey(), next.getValue());
+                     }
+                 }
+                 catch (IgniteCheckedException e) {
+                     if (!ctx.discovery().alive(oldestSrvNode) || !ctx.discovery().pingNode(oldestSrvNode.id()))
+                         continue;
+                     else
+                         throw e;
+                 }
+                 catch (CacheException e) {
+                     if (X.hasCause(e, ClusterTopologyCheckedException.class, ClusterTopologyException.class))
+                         continue;
+                     else
+                         throw e;
+                 }
+ 
+                 break;
+             }
+         }
+ 
+         for (Map.Entry<Integer, BinaryMetadata> e : metaBuf.entrySet())
+             addMeta(e.getKey(), e.getValue().wrap(binaryCtx));
+ 
+         metaBuf.clear();
+ 
+         startLatch.countDown();
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public void onKernalStop(boolean cancel) {
+         super.onKernalStop(cancel);
+ 
+         if (metaCacheQryId != null)
+             metaDataCache.context().continuousQueries().cancelInternalQuery(metaCacheQryId);
+     }
+ 
+     /**
+      * @param key Metadata key.
+      * @param newMeta Metadata.
+      */
+     private void addClientCacheMetaData(BinaryMetadataKey key, final BinaryMetadata newMeta) {
+         int key0 = key.typeId();
+ 
+         clientMetaDataCache.compute(key0, new ConcurrentHashMap8.BiFun<Integer, BinaryTypeImpl, BinaryTypeImpl>() {
+             @Override public BinaryTypeImpl apply(Integer key, BinaryTypeImpl oldMeta) {
+                 BinaryMetadata res;
+ 
+                 BinaryMetadata oldMeta0 = oldMeta != null ? oldMeta.metadata() : null;
+ 
+                 try {
+                     res = BinaryUtils.mergeMetadata(oldMeta0, newMeta);
+                 }
+                 catch (BinaryObjectException e) {
+                     res = oldMeta0;
+                 }
+ 
+                 return res != null ? res.wrap(binaryCtx) : null;
+             }
+         });
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public int typeId(String typeName) {
+         if (binaryCtx == null)
+             return super.typeId(typeName);
+ 
+         return binaryCtx.typeId(typeName);
+     }
+ 
+     /**
+      * @param obj Object.
+      * @return Bytes.
+      * @throws org.apache.ignite.binary.BinaryObjectException If failed.
+      */
+     public byte[] marshal(@Nullable Object obj) throws BinaryObjectException {
+         byte[] arr = binaryMarsh.marshal(obj);
+ 
+         assert arr.length > 0;
+ 
+         return arr;
+     }
+ 
+     /**
+      * @param ptr Off-heap pointer.
+      * @param forceHeap If {@code true} creates heap-based object.
+      * @return Object.
+      * @throws org.apache.ignite.binary.BinaryObjectException If failed.
+      */
+     public Object unmarshal(long ptr, boolean forceHeap) throws BinaryObjectException {
+         assert ptr > 0 : ptr;
+ 
+         int size = UNSAFE.getInt(ptr);
+ 
+         ptr += 4;
+ 
+         byte type = UNSAFE.getByte(ptr++);
+ 
+         if (type != CacheObject.TYPE_BYTE_ARR) {
+             assert size > 0 : size;
+ 
+             BinaryInputStream in = new BinaryOffheapInputStream(ptr, size, forceHeap);
+ 
+             return binaryMarsh.unmarshal(in);
+         }
+         else
+             return U.copyMemory(ptr, size);
+     }
+ 
+     /** {@inheritDoc} */
+     @SuppressWarnings("unchecked")
+     @Override public Object marshalToBinary(@Nullable Object obj) throws BinaryObjectException {
+         if (obj == null)
+             return null;
+ 
+         if (BinaryUtils.isBinaryType(obj.getClass()))
+             return obj;
+ 
+         if (obj instanceof Object[]) {
+             Object[] arr = (Object[])obj;
+ 
+             Object[] pArr = new Object[arr.length];
+ 
+             for (int i = 0; i < arr.length; i++)
+                 pArr[i] = marshalToBinary(arr[i]);
+ 
+             return pArr;
+         }
+ 
+         if (obj instanceof IgniteBiTuple) {
+             IgniteBiTuple tup = (IgniteBiTuple)obj;
+ 
+             if (obj instanceof T2)
+                 return new T2<>(marshalToBinary(tup.get1()), marshalToBinary(tup.get2()));
+ 
+             return new IgniteBiTuple<>(marshalToBinary(tup.get1()), marshalToBinary(tup.get2()));
+         }
+ 
+         if (obj instanceof Collection) {
+             Collection<Object> col = (Collection<Object>)obj;
+ 
+             Collection<Object> pCol;
+ 
+             if (col instanceof Set)
+                 pCol = (Collection<Object>)BinaryUtils.newSet((Set<?>)col);
+             else
+                 pCol = new ArrayList<>(col.size());
+ 
+             for (Object item : col)
+                 pCol.add(marshalToBinary(item));
+ 
+             return pCol;
+         }
+ 
+         if (obj instanceof Map) {
+             Map<?, ?> map = (Map<?, ?>)obj;
+ 
+             Map<Object, Object> pMap = BinaryUtils.newMap((Map<Object, Object>)obj);
+ 
+             for (Map.Entry<?, ?> e : map.entrySet())
+                 pMap.put(marshalToBinary(e.getKey()), marshalToBinary(e.getValue()));
+ 
+             return pMap;
+         }
+ 
+         if (obj instanceof Map.Entry) {
+             Map.Entry<?, ?> e = (Map.Entry<?, ?>)obj;
+ 
+             return new GridMapEntry<>(marshalToBinary(e.getKey()), marshalToBinary(e.getValue()));
+         }
+ 
+         byte[] arr = binaryMarsh.marshal(obj);
+ 
+         assert arr.length > 0;
+ 
+         Object obj0 = binaryMarsh.unmarshal(arr, null);
+ 
+         // Possible if a class has writeObject method.
+         if (obj0 instanceof BinaryObject)
+             ((BinaryObjectImpl)obj0).detachAllowed(true);
+ 
+         return obj0;
+     }
+ 
+     /**
+      * @return Marshaller.
+      */
+     public GridBinaryMarshaller marshaller() {
+         return binaryMarsh;
+     }
+ 
+     /** {@inheritDoc} */
++    @Override public String affinityField(String keyType) {
++        if (portableCtx == null)
++            return null;
++
++        return portableCtx.affinityKeyFieldName(typeId(keyType));
++    }
++
++    /** {@inheritDoc} */
+     @Override public BinaryObjectBuilder builder(String clsName) {
+         return new BinaryObjectBuilderImpl(binaryCtx, clsName);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public BinaryObjectBuilder builder(BinaryObject binaryObj) {
+         return BinaryObjectBuilderImpl.wrap(binaryObj);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public void updateMetadata(int typeId, String typeName, @Nullable String affKeyFieldName,
+         Map<String, Integer> fieldTypeIds, boolean isEnum) throws BinaryObjectException {
+         BinaryMetadata meta = new BinaryMetadata(typeId, typeName, fieldTypeIds, affKeyFieldName, null, isEnum);
+ 
+         binaryCtx.updateMetadata(typeId, meta);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public void addMeta(final int typeId, final BinaryType newMeta) throws BinaryObjectException {
+         assert newMeta != null;
+         assert newMeta instanceof BinaryTypeImpl;
+ 
+         BinaryMetadata newMeta0 = ((BinaryTypeImpl)newMeta).metadata();
+ 
+         final BinaryMetadataKey key = new BinaryMetadataKey(typeId);
+ 
+         try {
+             BinaryMetadata oldMeta = metaDataCache.localPeek(key);
+             BinaryMetadata mergedMeta = BinaryUtils.mergeMetadata(oldMeta, newMeta0);
+ 
+             BinaryObjectException err = metaDataCache.invoke(key, new MetadataProcessor(mergedMeta));
+ 
+             if (err != null)
+                 throw err;
+         }
+         catch (CacheException e) {
+             throw new BinaryObjectException("Failed to update meta data for type: " + newMeta.typeName(), e);
+         }
+     }
+ 
+     /** {@inheritDoc} */
+     @Nullable @Override public BinaryType metadata(final int typeId) throws BinaryObjectException {
+         try {
+             if (clientNode) {
+                 BinaryType typeMeta = clientMetaDataCache.get(typeId);
+ 
+                 if (typeMeta != null)
+                     return typeMeta;
+ 
+                 BinaryMetadata meta = metaDataCache.getTopologySafe(new BinaryMetadataKey(typeId));
+ 
+                 return meta != null ? meta.wrap(binaryCtx) : null;
+             }
+             else {
+                 BinaryMetadataKey key = new BinaryMetadataKey(typeId);
+ 
+                 BinaryMetadata meta = metaDataCache.localPeek(key);
+ 
+                 if (meta == null && !metaDataCache.context().preloader().syncFuture().isDone())
+                     meta = metaDataCache.getTopologySafe(key);
+ 
+                 return meta != null ? meta.wrap(binaryCtx) : null;
+             }
+         }
+         catch (CacheException e) {
+             throw new BinaryObjectException(e);
+         }
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public Map<Integer, BinaryType> metadata(Collection<Integer> typeIds)
+         throws BinaryObjectException {
+         try {
+             Collection<BinaryMetadataKey> keys = new ArrayList<>(typeIds.size());
+ 
+             for (Integer typeId : typeIds)
+                 keys.add(new BinaryMetadataKey(typeId));
+ 
+             Map<BinaryMetadataKey, BinaryMetadata> meta = metaDataCache.getAll(keys);
+ 
+             Map<Integer, BinaryType> res = U.newHashMap(meta.size());
+ 
+             for (Map.Entry<BinaryMetadataKey, BinaryMetadata> e : meta.entrySet())
+                 res.put(e.getKey().typeId(), e.getValue().wrap(binaryCtx));
+ 
+             return res;
+         }
+         catch (CacheException e) {
+             throw new BinaryObjectException(e);
+         }
+     }
+ 
+     /** {@inheritDoc} */
+     @SuppressWarnings("unchecked")
+     @Override public Collection<BinaryType> metadata() throws BinaryObjectException {
+         if (clientNode)
+             return F.viewReadOnly(clientMetaDataCache.values(), new IgniteClosure<BinaryTypeImpl, BinaryType>() {
+                 @Override public BinaryType apply(BinaryTypeImpl meta) {
+                     return meta;
+                 }
+             });
+         else {
+             return F.viewReadOnly(metaDataCache.entrySetx(metaPred),
+                 new C1<Cache.Entry<BinaryMetadataKey, BinaryMetadata>, BinaryType>() {
+                     private static final long serialVersionUID = 0L;
+ 
+                     @Override public BinaryType apply(Cache.Entry<BinaryMetadataKey, BinaryMetadata> e) {
+                         return e.getValue().wrap(binaryCtx);
+                     }
+                 });
+         }
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public BinaryObject buildEnum(String typeName, int ord) throws IgniteException {
+         typeName = BinaryContext.typeName(typeName);
+ 
+         int typeId = binaryCtx.typeId(typeName);
+ 
+         updateMetadata(typeId, typeName, null, null, true);
+ 
+         return new BinaryEnumObjectImpl(binaryCtx, typeId, null, ord);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public IgniteBinary binary() throws IgniteException {
+         return binaries;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public boolean isBinaryObject(Object obj) {
+         return obj instanceof BinaryObject;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public boolean isBinaryEnabled(CacheConfiguration<?, ?> ccfg) {
+         return marsh instanceof BinaryMarshaller;
+     }
+ 
+     /**
+      * @param po Binary object.
+      * @return Affinity key.
+      */
+     public Object affinityKey(BinaryObject po) {
+         try {
+             BinaryType meta = po.type();
+ 
+             if (meta != null) {
+                 String affKeyFieldName = meta.affinityKeyFieldName();
+ 
+                 if (affKeyFieldName != null)
+                     return po.field(affKeyFieldName);
+             }
+             else if (po instanceof BinaryObjectEx) {
+                 int id = ((BinaryObjectEx)po).typeId();
+ 
+                 String affKeyFieldName = binaryCtx.affinityKeyFieldName(id);
+ 
+                 if (affKeyFieldName != null)
+                     return po.field(affKeyFieldName);
+             }
+         }
+         catch (BinaryObjectException e) {
+             U.error(log, "Failed to get affinity field from binary object: " + po, e);
+         }
+ 
+         return po;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public int typeId(Object obj) {
+         if (obj == null)
+             return 0;
+ 
+         return isBinaryObject(obj) ? ((BinaryObjectEx)obj).typeId() : typeId(obj.getClass().getSimpleName());
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public Object field(Object obj, String fieldName) {
+         if (obj == null)
+             return null;
+ 
+         return isBinaryObject(obj) ? ((BinaryObject)obj).field(fieldName) : super.field(obj, fieldName);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public boolean hasField(Object obj, String fieldName) {
+         return obj != null && ((BinaryObject)obj).hasField(fieldName);
+     }
+ 
+     /**
+      * @return Binary context.
+      */
+     public BinaryContext binaryContext() {
+         return binaryCtx;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public CacheObjectContext contextForCache(CacheConfiguration cfg) throws IgniteCheckedException {
+         assert cfg != null;
+ 
+         boolean binaryEnabled = marsh instanceof BinaryMarshaller && !GridCacheUtils.isSystemCache(cfg.getName()) &&
+             !GridCacheUtils.isIgfsCache(ctx.config(), cfg.getName());
+ 
+         CacheObjectContext ctx0 = super.contextForCache(cfg);
+ 
+         CacheObjectContext res = new CacheObjectBinaryContext(ctx,
+             ctx0.copyOnGet(),
+             ctx0.storeValue(),
+             binaryEnabled,
+             ctx0.addDeploymentInfo());
+ 
+         ctx.resource().injectGeneric(res.defaultAffMapper());
+ 
+         return res;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public byte[] marshal(CacheObjectContext ctx, Object val) throws IgniteCheckedException {
+         if (!((CacheObjectBinaryContext)ctx).binaryEnabled() || binaryMarsh == null)
+             return super.marshal(ctx, val);
+ 
+         byte[] arr = binaryMarsh.marshal(val);
+ 
+         assert arr.length > 0;
+ 
+         return arr;
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public Object unmarshal(CacheObjectContext ctx, byte[] bytes, ClassLoader clsLdr)
+         throws IgniteCheckedException {
+         if (!((CacheObjectBinaryContext)ctx).binaryEnabled() || binaryMarsh == null)
+             return super.unmarshal(ctx, bytes, clsLdr);
+ 
+         return binaryMarsh.unmarshal(bytes, clsLdr);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public KeyCacheObject toCacheKeyObject(CacheObjectContext ctx, Object obj, boolean userObj) {
+         if (!((CacheObjectBinaryContext)ctx).binaryEnabled())
+             return super.toCacheKeyObject(ctx, obj, userObj);
+ 
+         if (obj instanceof KeyCacheObject)
+             return (KeyCacheObject)obj;
+ 
+         if (((CacheObjectBinaryContext)ctx).binaryEnabled()) {
+             obj = toBinary(obj);
+ 
+             if (obj instanceof BinaryObject)
+                 return (BinaryObjectImpl)obj;
+         }
+ 
+         return toCacheKeyObject0(obj, userObj);
+     }
+ 
+     /** {@inheritDoc} */
+     @Nullable @Override public CacheObject toCacheObject(CacheObjectContext ctx, @Nullable Object obj,
+         boolean userObj) {
+         if (!((CacheObjectBinaryContext)ctx).binaryEnabled())
+             return super.toCacheObject(ctx, obj, userObj);
+ 
+         if (obj == null || obj instanceof CacheObject)
+             return (CacheObject)obj;
+ 
+         obj = toBinary(obj);
+ 
+         if (obj instanceof BinaryObject)
+             return (BinaryObjectImpl)obj;
+ 
+         return toCacheObject0(obj, userObj);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public CacheObject toCacheObject(CacheObjectContext ctx, byte type, byte[] bytes) {
+         if (type == BinaryObjectImpl.TYPE_BINARY)
+             return new BinaryObjectImpl(binaryContext(), bytes, 0);
+ 
+         return super.toCacheObject(ctx, type, bytes);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public CacheObject toCacheObject(GridCacheContext ctx, long valPtr, boolean tmp)
+         throws IgniteCheckedException {
+         if (!((CacheObjectBinaryContext)ctx.cacheObjectContext()).binaryEnabled())
+             return super.toCacheObject(ctx, valPtr, tmp);
+ 
+         Object val = unmarshal(valPtr, !tmp);
+ 
+         if (val instanceof BinaryObjectOffheapImpl)
+             return (BinaryObjectOffheapImpl)val;
+ 
+         return new CacheObjectImpl(val, null);
+     }
+ 
+     /** {@inheritDoc} */
+     @Override public Object unwrapTemporary(GridCacheContext ctx, Object obj) throws BinaryObjectException {
+         if (!((CacheObjectBinaryContext)ctx.cacheObjectContext()).binaryEnabled())
+             return obj;
+ 
+         if (obj instanceof BinaryObjectOffheapImpl)
+             return ((BinaryObjectOffheapImpl)obj).heapCopy();
+ 
+         return obj;
+     }
+ 
+     /**
+      * @param obj Object.
+      * @return Binary object.
+      * @throws IgniteException In case of error.
+      */
+     @Nullable public Object toBinary(@Nullable Object obj) throws IgniteException {
+         if (obj == null)
+             return null;
+ 
+         if (isBinaryObject(obj))
+             return obj;
+ 
+         return marshalToBinary(obj);
+     }
+ 
+     /**
+      * Processor responsible for metadata update.
+      */
+     private static class MetadataProcessor
+         implements EntryProcessor<BinaryMetadataKey, BinaryMetadata, BinaryObjectException>, Externalizable {
+         /** */
+         private static final long serialVersionUID = 0L;
+ 
+         /** */
+         private BinaryMetadata newMeta;
+ 
+         /**
+          * For {@link Externalizable}.
+          */
+         public MetadataProcessor() {
+             // No-op.
+         }
+ 
+         /**
+          * @param newMeta New metadata.
+          */
+         private MetadataProcessor(BinaryMetadata newMeta) {
+             assert newMeta != null;
+ 
+             this.newMeta = newMeta;
+         }
+ 
+         /** {@inheritDoc} */
+         @Override public BinaryObjectException process(MutableEntry<BinaryMetadataKey, BinaryMetadata> entry,
+             Object... args) {
+             try {
+                 BinaryMetadata oldMeta = entry.getValue();
+ 
+                 BinaryMetadata mergedMeta = BinaryUtils.mergeMetadata(oldMeta, newMeta);
+ 
+                 if (mergedMeta != oldMeta)
+                     entry.setValue(mergedMeta);
+ 
+                 return null;
+             }
+             catch (BinaryObjectException e) {
+                 return e;
+             }
+         }
+ 
+         /** {@inheritDoc} */
+         @Override public void writeExternal(ObjectOutput out) throws IOException {
+             out.writeObject(newMeta);
+         }
+ 
+         /** {@inheritDoc} */
+         @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+             newMeta = (BinaryMetadata)in.readObject();
+         }
+ 
+         /** {@inheritDoc} */
+         @Override public String toString() {
+             return S.toString(MetadataProcessor.class, this);
+         }
+     }
+ 
+     /**
+      *
+      */
+     class MetaDataEntryListener implements CacheEntryUpdatedListener<BinaryMetadataKey, BinaryMetadata> {
+         /** {@inheritDoc} */
+         @Override public void onUpdated(
+             Iterable<CacheEntryEvent<? extends BinaryMetadataKey, ? extends BinaryMetadata>> evts)
+             throws CacheEntryListenerException {
+             for (CacheEntryEvent<? extends BinaryMetadataKey, ? extends BinaryMetadata> evt : evts) {
+                 assert evt.getEventType() == EventType.CREATED || evt.getEventType() == EventType.UPDATED : evt;
+ 
+                 BinaryMetadataKey key = evt.getKey();
+ 
+                 final BinaryMetadata newMeta = evt.getValue();
+ 
+                 assert newMeta != null : evt;
+ 
+                 addClientCacheMetaData(key, newMeta);
+             }
+         }
+ 
+         /** {@inheritDoc} */
+         @Override public String toString() {
+             return S.toString(MetaDataEntryListener.class, this);
+         }
+     }
+ 
+     /**
+      *
+      */
+     static class MetaDataEntryFilter implements CacheEntryEventSerializableFilter<Object, Object> {
+         /** */
+         private static final long serialVersionUID = 0L;
+ 
+         /** {@inheritDoc} */
+         @Override public boolean evaluate(CacheEntryEvent<?, ?> evt) throws CacheEntryListenerException {
+             return evt.getKey() instanceof BinaryMetadataKey;
+         }
+ 
+         /** {@inheritDoc} */
+         @Override public String toString() {
+             return S.toString(MetaDataEntryFilter.class, this);
+         }
+     }
+ 
+     /**
+      *
+      */
+     static class MetaDataPredicate implements IgniteBiPredicate<Object, Object> {
+         /** */
+         private static final long serialVersionUID = 0L;
+ 
+         /** {@inheritDoc} */
+         @Override public boolean apply(Object key, Object val) {
+             return key instanceof BinaryMetadataKey;
+         }
+ 
+         /** {@inheritDoc} */
+         @Override public String toString() {
+             return S.toString(MetaDataPredicate.class, this);
+         }
+     }
+ }

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddb817b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
index e552411,cadf1a9..27d93ad
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
@@@ -170,10 -170,4 +170,10 @@@ public interface IgniteCacheObjectProce
       * @return Ignite binary interface.
       */
      public IgniteBinary binary();
 +
 +    /**
 +     * @param keyType Key type name.
 +     * @return Affinity filed name or {@code null}.
 +     */
 +    public String affinityField(String keyType);
- }
+ }

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddb817b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddb817b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 89bf44c,8625be9..7b696ff
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@@ -1113,8 -1045,7 +1116,8 @@@ public class IgniteH2Indexing implement
  
          twoStepQry.pageSize(qry.getPageSize());
  
 -        QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(queryTwoStep(cctx, twoStepQry, cctx.keepBinary()));
 +        QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(
-             runQueryTwoStep(cctx, twoStepQry, cctx.keepPortable(), enforceJoinOrder));
++            runQueryTwoStep(cctx, twoStepQry, cctx.keepBinary(), enforceJoinOrder));
  
          cursor.fieldsMeta(meta);
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddb817b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 6e9bf21,828d9bd..00d2f27
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@@ -1350,24 -1240,4 +1350,24 @@@ public class GridReduceQueryExecutor 
              return res;
          }
      }
 +
 +    /**
 +     *
 +     */
 +    private class ExplicitPartitionsSpecializer implements IgniteBiClosure<ClusterNode,Message,Message> {
 +        /** */
 +        private final Map<ClusterNode,IntArray> partsMap;
 +
 +        /**
 +         * @param partsMap Partitions map.
 +         */
 +        private ExplicitPartitionsSpecializer(Map<ClusterNode,IntArray> partsMap) {
 +            this.partsMap = partsMap;
 +        }
 +
 +        /** {@inheritDoc} */
 +        @Override public Message apply(ClusterNode n, Message msg) {
 +            return copy(msg, n, partsMap);
 +        }
 +    }
- }
+ }

http://git-wip-us.apache.org/repos/asf/ignite/blob/dddb817b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
----------------------------------------------------------------------


[12/50] [abbrv] ignite git commit: ignite-split2 - regen

Posted by sb...@apache.org.
ignite-split2 - regen


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

Branch: refs/heads/ignite-1232
Commit: d87978be970d8a5945ea77302a8fce62e2acab70
Parents: 91c7b8f
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Dec 7 09:02:22 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Dec 7 09:02:22 2015 +0300

----------------------------------------------------------------------
 .../h2/twostep/msg/GridH2QueryRequest.java      | 141 ++++++++++++++++++-
 1 file changed, 138 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d87978be/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java
index 00c7ef9..bd932f9 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java
@@ -32,6 +32,7 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 
@@ -76,6 +77,8 @@ public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable {
     private byte flags;
 
     /** */
+    @GridToStringInclude
+    @GridDirectCollection(String.class)
     private Collection<String> tbls;
 
     /**
@@ -235,12 +238,144 @@ public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable {
 
     /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
-        return false;
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeCollection("caches", caches, MessageCollectionItemType.STRING))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeByte("flags", flags))
+                    return false;
+
+                writer.incrementState();
+
+            case 2:
+                if (!writer.writeInt("pageSize", pageSize))
+                    return false;
+
+                writer.incrementState();
+
+            case 3:
+                if (!writer.writeMap("parts", parts, MessageCollectionItemType.UUID, MessageCollectionItemType.INT_ARR))
+                    return false;
+
+                writer.incrementState();
+
+            case 4:
+                if (!writer.writeCollection("qrys", qrys, MessageCollectionItemType.MSG))
+                    return false;
+
+                writer.incrementState();
+
+            case 5:
+                if (!writer.writeLong("reqId", reqId))
+                    return false;
+
+                writer.incrementState();
+
+            case 6:
+                if (!writer.writeCollection("tbls", tbls, MessageCollectionItemType.STRING))
+                    return false;
+
+                writer.incrementState();
+
+            case 7:
+                if (!writer.writeMessage("topVer", topVer))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
     }
 
     /** {@inheritDoc} */
     @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
-        return false;
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                caches = reader.readCollection("caches", MessageCollectionItemType.STRING);
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                flags = reader.readByte("flags");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 2:
+                pageSize = reader.readInt("pageSize");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 3:
+                parts = reader.readMap("parts", MessageCollectionItemType.UUID, MessageCollectionItemType.INT_ARR, false);
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 4:
+                qrys = reader.readCollection("qrys", MessageCollectionItemType.MSG);
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 5:
+                reqId = reader.readLong("reqId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 6:
+                tbls = reader.readCollection("tbls", MessageCollectionItemType.STRING);
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 7:
+                topVer = reader.readMessage("topVer");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(GridH2QueryRequest.class);
     }
 
     /** {@inheritDoc} */
@@ -250,7 +385,7 @@ public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 0;
+        return 8;
     }
 
     /** {@inheritDoc} */


[45/50] [abbrv] ignite git commit: ignite-split2 - dropped GridClosurePolicy

Posted by sb...@apache.org.
ignite-split2 - dropped GridClosurePolicy


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

Branch: refs/heads/ignite-1232
Commit: 94834737db81a820472dd1feeb1a9b15069d7949
Parents: e448d07
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Jan 11 15:00:29 2016 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Jan 11 15:00:29 2016 +0300

----------------------------------------------------------------------
 .../managers/communication/GridIoManager.java   | 12 ++++-
 .../managers/communication/GridIoPolicy.java    |  3 ++
 .../processors/closure/GridClosurePolicy.java   | 51 -------------------
 .../closure/GridClosureProcessor.java           | 52 ++++++--------------
 .../processors/igfs/IgfsIpcHandler.java         |  4 +-
 5 files changed, 31 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/94834737/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index 42f8dae..ec2d797 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -87,6 +87,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.internal.GridTopic.TOPIC_COMM_USER;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.AFFINITY_POOL;
+import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IGFS_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.MANAGEMENT_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.MARSH_CACHE_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.P2P_POOL;
@@ -143,6 +144,9 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
     /** Marshaller cache pool. */
     private ExecutorService marshCachePool;
 
+    /** IGFS pool. */
+    private ExecutorService igfsPool;
+
     /** Discovery listener. */
     private GridLocalEventListener discoLsnr;
 
@@ -241,6 +245,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
         mgmtPool = ctx.getManagementExecutorService();
         utilityCachePool = ctx.utilityCachePool();
         marshCachePool = ctx.marshallerCachePool();
+        igfsPool = ctx.getIgfsExecutorService();
         affPool = new IgniteThreadPoolExecutor(
             "aff",
             ctx.gridName(),
@@ -668,7 +673,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
      * @return Execution pool.
      * @throws IgniteCheckedException If failed.
      */
-    private Executor pool(byte plc) throws IgniteCheckedException {
+    public Executor pool(byte plc) throws IgniteCheckedException {
         switch (plc) {
             case P2P_POOL:
                 return p2pPool;
@@ -691,6 +696,11 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
 
                 return marshCachePool;
 
+            case IGFS_POOL:
+                assert igfsPool != null : "IGFS pool is not configured.";
+
+                return igfsPool;
+
             default: {
                 assert plc >= 0 : "Negative policy: " + plc;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/94834737/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java
index 57622c9..a417857 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java
@@ -43,6 +43,9 @@ public class GridIoPolicy {
     /** Marshaller cache execution pool. */
     public static final byte MARSH_CACHE_POOL = 6;
 
+    /** IGFS pool. */
+    public static final byte IGFS_POOL = 7;
+
     /**
      * Defines the range of reserved pools that are not available for plugins.
      * @param key The key.

http://git-wip-us.apache.org/repos/asf/ignite/blob/94834737/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosurePolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosurePolicy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosurePolicy.java
deleted file mode 100644
index c17cedd..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosurePolicy.java
+++ /dev/null
@@ -1,51 +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.closure;
-
-import org.jetbrains.annotations.Nullable;
-
-/**
- * This enumeration defines different types of closure
- * processing by the closure processor.
- */
-public enum GridClosurePolicy {
-    /** Public execution pool. */
-    PUBLIC_POOL,
-
-    /** P2P execution pool. */
-    P2P_POOL,
-
-    /** System execution pool. */
-    SYSTEM_POOL,
-
-    /** IGFS pool. */
-    IGFS_POOL;
-
-    /** Enum values. */
-    private static final GridClosurePolicy[] VALS = values();
-
-    /**
-     * Efficiently gets enumerated value from its ordinal.
-     *
-     * @param ord Ordinal value.
-     * @return Enumerated value.
-     */
-    @Nullable public static GridClosurePolicy fromOrdinal(int ord) {
-        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/94834737/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
index b4d79d9..c53cb8b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
@@ -69,6 +69,9 @@ import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.compute.ComputeJobResultPolicy.FAILOVER;
 import static org.apache.ignite.compute.ComputeJobResultPolicy.REDUCE;
+import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IGFS_POOL;
+import static org.apache.ignite.internal.managers.communication.GridIoPolicy.PUBLIC_POOL;
+import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
 import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_NO_FAILOVER;
 import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_SUBGRID;
 
@@ -76,15 +79,6 @@ import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKe
  *
  */
 public class GridClosureProcessor extends GridProcessorAdapter {
-    /** */
-    private final Executor sysPool;
-
-    /** */
-    private final Executor pubPool;
-
-    /** */
-    private final Executor igfsPool;
-
     /** Lock to control execution after stop. */
     private final GridSpinReadWriteLock busyLock = new GridSpinReadWriteLock();
 
@@ -96,10 +90,6 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      */
     public GridClosureProcessor(GridKernalContext ctx) {
         super(ctx);
-
-        sysPool = ctx.getSystemExecutorService();
-        pubPool = ctx.getExecutorService();
-        igfsPool = ctx.getIgfsExecutorService();
     }
 
     /** {@inheritDoc} */
@@ -712,20 +702,8 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param plc Whether to get system or public pool.
      * @return Requested worker pool.
      */
-    private Executor pool(GridClosurePolicy plc) {
-        switch (plc) {
-            case PUBLIC_POOL:
-                return pubPool;
-
-            case SYSTEM_POOL:
-                return sysPool;
-
-            case IGFS_POOL:
-                return igfsPool;
-
-            default:
-                throw new IllegalArgumentException("Invalid closure execution policy: " + plc);
-        }
+    private Executor pool(byte plc) throws IgniteCheckedException {
+        return ctx.io().pool(plc);
     }
 
     /**
@@ -734,7 +712,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param plc Policy to choose executor pool.
      * @return Pool name.
      */
-    private String poolName(GridClosurePolicy plc) {
+    private String poolName(byte plc) {
         switch (plc) {
             case PUBLIC_POOL:
                 return "public";
@@ -746,7 +724,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
                 return "igfs";
 
             default:
-                throw new IllegalArgumentException("Invalid closure execution policy: " + plc);
+                return "unknown";
         }
     }
 
@@ -757,7 +735,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
     private IgniteInternalFuture<?> runLocal(@Nullable final Runnable c, boolean sys) throws IgniteCheckedException {
-        return runLocal(c, sys ? GridClosurePolicy.SYSTEM_POOL : GridClosurePolicy.PUBLIC_POOL);
+        return runLocal(c, sys ? SYSTEM_POOL : PUBLIC_POOL);
     }
 
     /**
@@ -766,7 +744,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @return Future.
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
-    private IgniteInternalFuture<?> runLocal(@Nullable final Runnable c, GridClosurePolicy plc) throws IgniteCheckedException {
+    private IgniteInternalFuture<?> runLocal(@Nullable final Runnable c, byte plc) throws IgniteCheckedException {
         if (c == null)
             return new GridFinishedFuture();
 
@@ -842,7 +820,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @return Future.
      */
     public IgniteInternalFuture<?> runLocalSafe(Runnable c, boolean sys) {
-        return runLocalSafe(c, sys ? GridClosurePolicy.SYSTEM_POOL : GridClosurePolicy.PUBLIC_POOL);
+        return runLocalSafe(c, sys ? SYSTEM_POOL : PUBLIC_POOL);
     }
 
     /**
@@ -853,7 +831,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param plc Policy to choose executor pool.
      * @return Future.
      */
-    public IgniteInternalFuture<?> runLocalSafe(Runnable c, GridClosurePolicy plc) {
+    public IgniteInternalFuture<?> runLocalSafe(Runnable c, byte plc) {
         try {
             return runLocal(c, plc);
         }
@@ -897,7 +875,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
     private <R> IgniteInternalFuture<R> callLocal(@Nullable final Callable<R> c, boolean sys) throws IgniteCheckedException {
-        return callLocal(c, sys ? GridClosurePolicy.SYSTEM_POOL : GridClosurePolicy.PUBLIC_POOL);
+        return callLocal(c, sys ? SYSTEM_POOL : PUBLIC_POOL);
     }
 
     /**
@@ -907,7 +885,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @return Future.
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
-    private <R> IgniteInternalFuture<R> callLocal(@Nullable final Callable<R> c, GridClosurePolicy plc) throws IgniteCheckedException {
+    private <R> IgniteInternalFuture<R> callLocal(@Nullable final Callable<R> c, byte plc) throws IgniteCheckedException {
         if (c == null)
             return new GridFinishedFuture<>();
 
@@ -981,7 +959,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @return Future.
      */
     public <R> IgniteInternalFuture<R> callLocalSafe(Callable<R> c, boolean sys) {
-        return callLocalSafe(c, sys ? GridClosurePolicy.SYSTEM_POOL : GridClosurePolicy.PUBLIC_POOL);
+        return callLocalSafe(c, sys ? SYSTEM_POOL : PUBLIC_POOL);
     }
 
     /**
@@ -992,7 +970,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param plc Policy to choose executor pool.
      * @return Future.
      */
-    public <R> IgniteInternalFuture<R> callLocalSafe(Callable<R> c, GridClosurePolicy plc) {
+    public <R> IgniteInternalFuture<R> callLocalSafe(Callable<R> c, byte plc) {
         try {
             return callLocal(c, plc);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/94834737/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
index 0ed7c0d..e5c2e2c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
@@ -36,7 +36,7 @@ import org.apache.ignite.internal.igfs.common.IgfsIpcCommand;
 import org.apache.ignite.internal.igfs.common.IgfsMessage;
 import org.apache.ignite.internal.igfs.common.IgfsPathControlRequest;
 import org.apache.ignite.internal.igfs.common.IgfsStreamControlRequest;
-import org.apache.ignite.internal.processors.closure.GridClosurePolicy;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.lang.GridPlainCallable;
 import org.apache.ignite.internal.util.typedef.F;
@@ -143,7 +143,7 @@ class IgfsIpcHandler implements IgfsServerHandler {
                             // No need to pass data input for non-write-block commands.
                             return execute(ses, cmd, msg, null);
                         }
-                    }, GridClosurePolicy.IGFS_POOL);
+                    }, GridIoPolicy.IGFS_POOL);
                 }
             }
 


[29/50] [abbrv] ignite git commit: ignite-split2 - retry

Posted by sb...@apache.org.
ignite-split2 - retry


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

Branch: refs/heads/ignite-1232
Commit: dad374fac124f92c3477990e114bb0dc9b860d52
Parents: 5cf4edc
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Dec 15 06:00:40 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Dec 15 06:00:40 2015 +0300

----------------------------------------------------------------------
 .../processors/query/h2/twostep/GridMapQueryExecutor.java    | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dad374fa/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index 6e1aef7..ec54a27 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -55,6 +55,7 @@ import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshalla
 import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryContext;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RetryException;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest;
 import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse;
@@ -66,6 +67,7 @@ import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.h2.jdbc.JdbcResultSet;
@@ -589,6 +591,12 @@ public class GridMapQueryExecutor {
      */
     private void sendError(ClusterNode node, long qryReqId, Throwable err) {
         try {
+            if (X.hasCause(err, GridH2RetryException.class)) {
+                sendRetry(node, qryReqId);
+
+                return;
+            }
+
             GridQueryFailResponse msg = new GridQueryFailResponse(qryReqId, err);
 
             if (node.isLocal())


[11/50] [abbrv] ignite git commit: ignite-split2 - row

Posted by sb...@apache.org.
ignite-split2 - row


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

Branch: refs/heads/ignite-1232
Commit: 91c7b8faf6b25037dda9faf94eb4e25db67ae622
Parents: 7a1d607
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Dec 7 08:16:44 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Dec 7 08:16:44 2015 +0300

----------------------------------------------------------------------
 .../query/h2/opt/GridH2AbstractKeyValueRow.java | 44 +++++++++++++++++---
 .../processors/query/h2/opt/GridH2Table.java    | 11 ++++-
 2 files changed, 48 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/91c7b8fa/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java
index 297e284..a15b569 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2AbstractKeyValueRow.java
@@ -61,6 +61,9 @@ public abstract class GridH2AbstractKeyValueRow extends GridH2Row {
     /** */
     private volatile Value val;
 
+    /** */
+    private Value[] valCache;
+
     /**
      * Constructor.
      *
@@ -196,6 +199,15 @@ public abstract class GridH2AbstractKeyValueRow extends GridH2Row {
 
     /** {@inheritDoc} */
     @Override public Value getValue(int col) {
+        Value[] vCache = valCache;
+
+        if (vCache != null) {
+            Value v = vCache[col];
+
+            if (v != null)
+                return v;
+        }
+
         if (col < DEFAULT_COLUMNS_COUNT) {
             Value v;
 
@@ -295,15 +307,35 @@ public abstract class GridH2AbstractKeyValueRow extends GridH2Row {
 
         Object res = desc.columnValue(key.getObject(), val.getObject(), col);
 
-        if (res == null)
-            return ValueNull.INSTANCE;
+        Value v;
 
-        try {
-            return desc.wrap(res, desc.fieldType(col));
+        if (res == null)
+            v = ValueNull.INSTANCE;
+        else {
+            try {
+                v = desc.wrap(res, desc.fieldType(col));
+            }
+            catch (IgniteCheckedException e) {
+                throw DbException.convert(e);
+            }
         }
-        catch (IgniteCheckedException e) {
-            throw DbException.convert(e);
+
+        if (vCache != null)
+            vCache[col + DEFAULT_COLUMNS_COUNT] = v;
+
+        return v;
+    }
+
+    /**
+     * @param valCache Value cache.
+     */
+    public void valuesCache(Value[] valCache) {
+        if (valCache != null) {
+            valCache[KEY_COL] = key;
+            valCache[VAL_COL] = val;
         }
+
+        this.valCache = valCache;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/91c7b8fa/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
index 17385e6..8ee00b9 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
@@ -463,7 +463,16 @@ public class GridH2Table extends TableBase {
 
         GridH2Row row = desc.createRow(key, val, expirationTime);
 
-        return doUpdate(row, rmv);
+        if (!rmv)
+            ((GridH2AbstractKeyValueRow)row).valuesCache(new Value[getColumns().length]);
+
+        try {
+            return doUpdate(row, rmv);
+        }
+        finally {
+            if (!rmv)
+                ((GridH2AbstractKeyValueRow)row).valuesCache(null);
+        }
     }
 
     /**


[06/50] [abbrv] ignite git commit: ignite-split2 - refactor

Posted by sb...@apache.org.
ignite-split2 - refactor


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

Branch: refs/heads/ignite-1232
Commit: b3b0312aa48725603f462f020ff58df2a82fdc07
Parents: 5c4f8b2
Author: S.Vladykin <sv...@gridgain.com>
Authored: Sun Dec 6 02:15:43 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Sun Dec 6 02:15:43 2015 +0300

----------------------------------------------------------------------
 .../query/h2/opt/GridH2Collocation.java         | 265 +++++++------------
 1 file changed, 99 insertions(+), 166 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b3b0312a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java
index 10cd0fe..c4cc6a5 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java
@@ -64,9 +64,6 @@ public final class GridH2Collocation {
     private TableFilter[] childFilters;
 
     /** */
-    private boolean childsOrderFinalized;
-
-    /** */
     private List<GridH2Collocation> unions;
 
     /** */
@@ -76,23 +73,36 @@ public final class GridH2Collocation {
      * @param upper Upper.
      * @param filter Filter.
      */
-    public GridH2Collocation(GridH2Collocation upper, int filter) {
+    private GridH2Collocation(GridH2Collocation upper, int filter) {
         this.upper = upper;
         this.filter = filter;
     }
 
     /**
-     * @return List of unions.
-     */
-    public List<GridH2Collocation> unions() {
-        return unions;
-    }
-
-    /**
+     * @param upper Upper.
+     * @param filter Filter.
      * @param unions Unions.
+     * @return Create collocation model.
      */
-    public void unions(List<GridH2Collocation> unions) {
-        this.unions = unions;
+    private static GridH2Collocation createChild(GridH2Collocation upper, int filter, List<GridH2Collocation> unions) {
+        GridH2Collocation child = new GridH2Collocation(upper, filter);
+
+        if (unions != null) {
+            // Bind created child to unions.
+            assert upper == null || upper.child(filter, false) != null;
+
+            unions.add(child);
+
+            child.unions = unions;
+        }
+        else if (upper != null) {
+            // Bind created child to upper model.
+            assert upper.child(filter, false) == null;
+
+            upper.children[filter] = child;
+        }
+
+        return child;
     }
 
     /**
@@ -114,8 +124,6 @@ public final class GridH2Collocation {
         else if (Arrays.equals(this.childFilters, childFilters))
             return false;
 
-        childsOrderFinalized = false;
-
         if (this.childFilters == null) {
             // We have to clone because H2 reuses array and reorders elements.
             this.childFilters = childFilters.clone();
@@ -131,17 +139,11 @@ public final class GridH2Collocation {
             Arrays.fill(children, null);
         }
 
-        reset();
-
-        return true;
-    }
-
-    /**
-     * Reset current collocation model and all the children, but do not touch union.
-     */
-    private void reset() {
+        // Reset results.
         type = null;
         multiplier = 0;
+
+        return true;
     }
 
     /**
@@ -149,13 +151,13 @@ public final class GridH2Collocation {
      * @param f Table filter.
      * @return {@code true} If the child is not a table or view.
      */
-    private boolean isNotTableOrViewChild(int i, TableFilter f) {
+    private boolean isChildTableOrView(int i, TableFilter f) {
         if (f == null)
             f = childFilters[i];
 
         Table t = f.getTable();
 
-        return !t.isView() && !(t instanceof GridH2Table);
+        return t.isView() || t instanceof GridH2Table;
     }
 
     /**
@@ -169,34 +171,33 @@ public final class GridH2Collocation {
             // We are at sub-query.
             boolean collocated = true;
             boolean partitioned = false;
-            int maxMultiplier = 0;
+            int maxMultiplier = MULTIPLIER_COLLOCATED;
 
             for (int i = 0; i < childFilters.length; i++) {
-                GridH2Collocation c = child(i);
+                GridH2Collocation c = child(i, true);
 
-                if (c == null) {
-                    assert isNotTableOrViewChild(i, null);
+                Type t = c.type(true);
 
-                    continue;
-                }
+                if (t.isPartitioned()) {
+                    partitioned = true;
 
-                Type t = c.type(true);
+                    if (!t.isCollocated()) {
+                        collocated = false;
 
-                if (!t.isCollocated()) {
-                    collocated = false;
+                        int m = c.multiplier(true);
 
-                    int m = c.multiplier(true);
+                        if (m > maxMultiplier) {
+                            maxMultiplier = m;
 
-                    if (m > maxMultiplier)
-                        maxMultiplier = m;
+                            if (maxMultiplier == MULTIPLIER_BROADCAST)
+                                break;
+                        }
+                    }
                 }
-
-                if (t.isPartitioned())
-                    partitioned = true;
             }
 
             type = Type.of(partitioned, collocated);
-            multiplier = type.isCollocated() ? MULTIPLIER_COLLOCATED : maxMultiplier;
+            multiplier = maxMultiplier;
         }
         else {
             assert upper != null;
@@ -242,6 +243,9 @@ public final class GridH2Collocation {
                     multiplier = MULTIPLIER_BROADCAST;
 
                     break;
+
+                default:
+                    throw new IllegalStateException();
             }
         }
     }
@@ -252,11 +256,9 @@ public final class GridH2Collocation {
      */
     private boolean findPartitionedTableBefore(int f) {
         for (int i = 0; i < f; i++) {
-            GridH2Collocation c = child(i);
-
-            assert c != null || isNotTableOrViewChild(i, null);
+            GridH2Collocation c = child(i, true);
 
-            // The `c` can be null if it is not a GridH2Table and not a sub-query,
+            // The c can be null if it is not a GridH2Table and not a sub-query,
             // it is a some kind of function table or anything else that considered replicated.
             if (c != null && c.type(true).isPartitioned())
                 return true;
@@ -297,9 +299,7 @@ public final class GridH2Collocation {
                     TableFilter prevJoin = expCol.getTableFilter();
 
                     if (prevJoin != null) {
-                        GridH2Collocation co = child(indexOf(prevJoin));
-
-                        assert co != null || isNotTableOrViewChild(-1, prevJoin);
+                        GridH2Collocation co = child(indexOf(prevJoin), true);
 
                         if (co != null) {
                             Type t = co.type(true);
@@ -342,7 +342,7 @@ public final class GridH2Collocation {
         Table t = col.getTable();
 
         if (t.isView()) {
-            Query qry = ((ViewIndex)f.getIndex()).getQuery();
+            Query qry = getSubQuery(f);
 
             return isAffinityColumn(qry, expCol);
         }
@@ -375,69 +375,9 @@ public final class GridH2Collocation {
     }
 
     /**
-     * Sets table filters to the final state of query.
-     *
-     * @return {@code false} if nothing was actually done here.
-     */
-    private boolean finalizeChildFiltersOrder() {
-        if (childFilters == null || childsOrderFinalized)
-            return false;
-
-        int i = 0;
-
-        // Collect table filters in final order after optimization.
-        for (TableFilter f = select.getTopTableFilter(); f != null; f = f.getJoin()) {
-            childFilters[i] = f;
-
-            GridH2Collocation c = child(i);
-
-            if (c == null)
-                child(i, c = new GridH2Collocation(this, i));
-
-            if (f.getTable().isView())
-                c.finalizeChildFiltersOrder();
-
-            i++;
-        }
-
-        assert i == childFilters.length;
-
-        reset();
-
-        childsOrderFinalized = true;
-
-        return true;
-    }
-
-    /**
      * @return Multiplier.
      */
     public int calculateMultiplier() {
-        if (childFilters != null && !childsOrderFinalized) {
-            // We have to set all sub-queries structure to the final one we will have in query.
-            boolean needReset = false;
-
-            for (int i = 0; i < childFilters.length; i++) {
-                Table t = childFilters[i].getTable();
-
-                if (t.isView() || t instanceof GridH2Table) {
-                    if (child(i) == null) {
-                        child(i, new GridH2Collocation(this, i));
-
-                        needReset = true;
-                    }
-
-                    if (t.isView() && child(i).finalizeChildFiltersOrder())
-                        needReset = true;
-                }
-            }
-
-            if (needReset)
-                reset();
-
-            childsOrderFinalized = true;
-        }
-
         // We don't need multiplier for union here because it will be summarized in H2.
         return multiplier(false);
     }
@@ -452,9 +392,9 @@ public final class GridH2Collocation {
         assert multiplier != 0;
 
         if (withUnion && unions != null) {
-            int maxMultiplier = unions.get(0).multiplier(false);
+            int maxMultiplier = 0;
 
-            for (int i = 1; i < unions.size(); i++) {
+            for (int i = 0; i < unions.size(); i++) {
                 int m = unions.get(i).multiplier(false);
 
                 if (m > maxMultiplier)
@@ -500,21 +440,43 @@ public final class GridH2Collocation {
     }
 
     /**
-     * @param idx Index.
-     * @param child Child collocation.
+     * @param i Index.
+     * @param create Create child if needed.
+     * @return Child collocation.
      */
-    private void child(int idx, GridH2Collocation child) {
-        assert children[idx] == null;
+    private GridH2Collocation child(int i, boolean create) {
+        GridH2Collocation child = children[i];
+
+        if (child == null && create && isChildTableOrView(i, null)) {
+            TableFilter f = childFilters[i];
 
-        children[idx] = child;
+            children[i] = child = f.getTable().isView() ?
+                buildCollocationModel(this, i, getSubQuery(f), null) :
+                createChild(this, i, null);
+        }
+
+        return child;
     }
 
     /**
-     * @param idx Index.
-     * @return Child collocation.
+     * @param f Table filter.
+     * @return Sub-query.
      */
-    private GridH2Collocation child(int idx) {
-        return children[idx];
+    private static Query getSubQuery(TableFilter f) {
+        return ((ViewIndex)f.getIndex()).getQuery();
+    }
+
+    /**
+     * @return Unions list.
+     */
+    private List<GridH2Collocation> getOrCreateUnions() {
+        if (unions == null) {
+            unions = new ArrayList<>(4);
+
+            unions.add(this);
+        }
+
+        return unions;
     }
 
     /**
@@ -537,7 +499,7 @@ public final class GridH2Collocation {
             c = qctx.queryCollocation();
 
             if (c == null) {
-                c = new GridH2Collocation(null, -1);
+                c = createChild(null, -1, null);
 
                 qctx.queryCollocation(c);
             }
@@ -548,48 +510,28 @@ public final class GridH2Collocation {
         // Handle union. We have to rely on fact that select will be the same on uppermost select.
         // For sub-queries we will drop collocation models, so that they will be recalculated anyways.
         if (c.select != null && c.select != select) {
-            List<GridH2Collocation> unions = c.unions();
-
-            int i = 1;
+            List<GridH2Collocation> unions = c.getOrCreateUnions();
 
-            if (unions == null) {
-                unions = new ArrayList<>();
-
-                unions.add(c);
-                c.unions(unions);
-            }
-            else {
-                for (; i < unions.size(); i++) {
-                    GridH2Collocation u = unions.get(i);
+            // Try to find this select in existing unions.
+            // Start with 1 because at 0 it always will be c.
+            for (int i = 1; i < unions.size(); i++) {
+                GridH2Collocation u = unions.get(i);
 
-                    if (u.select == select) {
-                        c = u;
+                if (u.select == select) {
+                    c = u;
 
-                        break;
-                    }
+                    break;
                 }
             }
 
-            if (i == unions.size()) {
-                c = new GridH2Collocation(c.upper, c.filter);
-
-                unions.add(c);
-
-                c.unions(unions);
-            }
+            // Nothing was found, need to create new child in union.
+            if (c.select != select)
+                c = createChild(c.upper, c.filter, unions);
         }
 
         c.childFilters(filters);
 
-        GridH2Collocation child = c.child(filter);
-
-        if (child == null) {
-            child = new GridH2Collocation(c, filter);
-
-            c.child(filter, child);
-        }
-
-        return child;
+        return c.child(filter, true);
     }
 
     /**
@@ -630,26 +572,17 @@ public final class GridH2Collocation {
 
         TableFilter[] filters = list.toArray(new TableFilter[list.size()]);
 
-        GridH2Collocation c = new GridH2Collocation(upper, filter);
-
-        if (unions != null) {
-            unions.add(c);
-
-            c.unions(unions);
-        }
+        GridH2Collocation c = createChild(upper, filter, unions);
 
         c.childFilters(filters);
 
-        if (upper != null)
-            upper.child(filter, c);
-
         for (int i = 0; i < filters.length; i++) {
             TableFilter f = filters[i];
 
             if (f.getTable().isView())
-                c.child(i, buildCollocationModel(c, i, ((ViewIndex)f.getIndex()).getQuery(), null));
+                buildCollocationModel(c, i, getSubQuery(f), null);
             else if (f.getTable() instanceof GridH2Table)
-                c.child(i, new GridH2Collocation(c, i));
+                createChild(c, i, null);
         }
 
         return upper == null ? c : null;


[48/50] [abbrv] ignite git commit: ignite-split2 - more reverts

Posted by sb...@apache.org.
ignite-split2 - more reverts


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

Branch: refs/heads/ignite-1232
Commit: 50fbeee3219feaf6e7b7e72f1e0b1d0c7dda7467
Parents: 4d8deba
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Jan 11 18:44:17 2016 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Jan 11 18:44:17 2016 +0300

----------------------------------------------------------------------
 .../h2/twostep/GridReduceQueryExecutor.java     | 24 +++++++++++---------
 1 file changed, 13 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/50fbeee3/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 3151677..8b3de90 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -524,9 +524,7 @@ public class GridReduceQueryExecutor {
 
             final String space = cctx.name();
 
-            List<GridCacheSqlQuery> mapQrys = qry.mapQueries();
-
-            final QueryRun r = new QueryRun(h2.connectionForSpace(space), mapQrys.size(), qry.pageSize());
+            final QueryRun r = new QueryRun(h2.connectionForSpace(space), qry.mapQueries().size(), qry.pageSize());
 
             AffinityTopologyVersion topVer = h2.readyTopologyVersion();
 
@@ -562,16 +560,18 @@ public class GridReduceQueryExecutor {
                 nodes = Collections.singleton(F.rand(nodes));
             }
 
-            final boolean skipMergeTbl = qry.skipMergeTable() && !qry.explain();
+            int tblIdx = 0;
+
+            final boolean skipMergeTbl = !qry.explain() && qry.skipMergeTable();
 
-            for (int i = 0; i < mapQrys.size(); i++) {
+            for (GridCacheSqlQuery mapQry : qry.mapQueries()) {
                 GridMergeIndex idx;
 
                 if (!skipMergeTbl) {
                     GridMergeTable tbl;
 
                     try {
-                        tbl = createMergeTable(r.conn, mapQrys.get(i), qry.explain());
+                        tbl = createMergeTable(r.conn, mapQry, qry.explain());
                     }
                     catch (IgniteCheckedException e) {
                         throw new IgniteException(e);
@@ -579,7 +579,7 @@ public class GridReduceQueryExecutor {
 
                     idx = tbl.getScanIndex(null);
 
-                    fakeTable(r.conn, i).innerTable(tbl);
+                    fakeTable(r.conn, tblIdx++).innerTable(tbl);
                 }
                 else
                     idx = GridMergeIndexUnsorted.createDummy();
@@ -589,7 +589,7 @@ public class GridReduceQueryExecutor {
                 r.idxs.add(idx);
             }
 
-            r.latch = new CountDownLatch(mapQrys.size() * nodes.size());
+            r.latch = new CountDownLatch(r.idxs.size() * nodes.size());
 
             runs.put(qryReqId, r);
 
@@ -600,6 +600,8 @@ public class GridReduceQueryExecutor {
                             "Client node disconnected."));
                 }
 
+                List<GridCacheSqlQuery> mapQrys = qry.mapQueries();
+
                 if (qry.explain()) {
                     mapQrys = new ArrayList<>(qry.mapQueries().size());
 
@@ -659,12 +661,12 @@ public class GridReduceQueryExecutor {
 
                 if (!retry) {
                     if (skipMergeTbl) {
+                        List<List<?>> res = new ArrayList<>();
+
                         assert r.idxs.size() == 1 : r.idxs;
 
                         GridMergeIndex idx = r.idxs.get(0);
 
-                        List<List<?>> res = new ArrayList<>((int)idx.getRowCountApproximation());
-
                         Cursor cur = idx.findInStream(null, null);
 
                         while (cur.next()) {
@@ -753,7 +755,7 @@ public class GridReduceQueryExecutor {
                     U.warn(log, "Query run was already removed: " + qryReqId);
 
                 if (!skipMergeTbl) {
-                    for (int i = 0; i < mapQrys.size(); i++)
+                    for (int i = 0, mapQrys = qry.mapQueries().size(); i < mapQrys; i++)
                         fakeTable(null, i).innerTable(null); // Drop all merge tables.
                 }
             }


[41/50] [abbrv] ignite git commit: ignite-split2 - compilation fix for H2 api

Posted by sb...@apache.org.
ignite-split2 - compilation fix for H2 api


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

Branch: refs/heads/ignite-1232
Commit: e0c86cddd80f6329ff71013164b88bbc13fc009b
Parents: e579c4c
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Dec 28 23:52:15 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Dec 28 23:52:15 2015 +0300

----------------------------------------------------------------------
 .../query/h2/opt/GridH2SpatialIndex.java        | 32 ++------------------
 .../processors/query/h2/opt/GridH2Table.java    |  7 +++--
 .../query/h2/opt/GridH2TreeIndex.java           |  2 +-
 .../query/h2/twostep/GridMergeIndex.java        |  3 +-
 4 files changed, 9 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e0c86cdd/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SpatialIndex.java
----------------------------------------------------------------------
diff --git a/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SpatialIndex.java b/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SpatialIndex.java
index a2dc7b3..b4618c1 100644
--- a/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SpatialIndex.java
+++ b/modules/geospatial/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2SpatialIndex.java
@@ -28,20 +28,18 @@ import java.util.Map;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.h2.engine.Constants;
 import org.h2.engine.Session;
 import org.h2.index.Cursor;
-import org.h2.index.IndexCondition;
 import org.h2.index.IndexType;
 import org.h2.index.SingleRowCursor;
 import org.h2.index.SpatialIndex;
+import org.h2.index.SpatialTreeIndex;
 import org.h2.message.DbException;
 import org.h2.mvstore.MVStore;
 import org.h2.mvstore.rtree.MVRTreeMap;
 import org.h2.mvstore.rtree.SpatialKey;
 import org.h2.result.SearchRow;
 import org.h2.result.SortOrder;
-import org.h2.table.Column;
 import org.h2.table.IndexColumn;
 import org.h2.table.Table;
 import org.h2.table.TableFilter;
@@ -231,32 +229,8 @@ public class GridH2SpatialIndex extends GridH2IndexBase implements SpatialIndex
     }
 
     /** {@inheritDoc} */
-    @Override protected long getCostRangeIndex(int[] masks, long rowCnt, TableFilter[] filters, int filter,
-        SortOrder sortOrder) {
-        rowCnt += Constants.COST_ROW_OFFSET;
-        long cost = rowCnt;
-        long rows = rowCnt;
-
-        if (masks == null)
-            return cost;
-
-        for (Column column : columns) {
-            int idx = column.getColumnId();
-            int mask = masks[idx];
-            if ((mask & IndexCondition.SPATIAL_INTERSECTS) != 0) {
-                cost = 3 + rows / 4;
-
-                break;
-            }
-        }
-
-        return cost;
-    }
-
-    /** {@inheritDoc} */
-    @Override public double getCost(Session ses, int[] masks, TableFilter[] filters, int filter,
-        SortOrder sortOrder) {
-        return getCostRangeIndex(masks, rowCnt, filters, filter, sortOrder);
+    @Override public double getCost(Session ses, int[] masks, TableFilter[] filters, int filter, SortOrder sortOrder) {
+        return SpatialTreeIndex.getCostRangeIndex(masks, rowCnt, columns);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0c86cdd/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
index ceb7922..f31f091 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
@@ -35,10 +35,10 @@ import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
 import org.h2.api.TableEngine;
 import org.h2.command.ddl.CreateTableData;
-import org.h2.engine.Constants;
 import org.h2.engine.Database;
 import org.h2.engine.DbObject;
 import org.h2.engine.Session;
+import org.h2.index.BaseIndex;
 import org.h2.index.Cursor;
 import org.h2.index.Index;
 import org.h2.index.IndexLookupBatch;
@@ -841,7 +841,7 @@ public class GridH2Table extends TableBase {
      * Wrapper type for primary key.
      */
     @SuppressWarnings("PackageVisibleInnerClass")
-    static class ScanIndex implements Index {
+    static class ScanIndex extends BaseIndex {
         /** */
         static final String SCAN_INDEX_NAME_SUFFIX = "__SCAN_";
 
@@ -934,9 +934,10 @@ public class GridH2Table extends TableBase {
         @Override public double getCost(Session ses, int[] masks, TableFilter[] filters, int filter,
             SortOrder sortOrder) {
             long rows = getRowCountApproximation();
+            double baseCost = getCostRangeIndex(masks, rows, filters, filter, sortOrder, true);
             int mul = delegate.getDistributedMultiplier(ses, filters, filter);
 
-            return  mul * (rows + Constants.COST_ROW_OFFSET);
+            return  mul * baseCost;
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0c86cdd/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index 1d4efff..f624292 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -464,7 +464,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
     /** {@inheritDoc} */
     @Override public double getCost(Session ses, int[] masks, TableFilter[] filters, int filter, SortOrder sortOrder) {
         long rowCnt = getRowCountApproximation();
-        double baseCost = getCostRangeIndex(masks, rowCnt, filters, filter, sortOrder);
+        double baseCost = getCostRangeIndex(masks, rowCnt, filters, filter, sortOrder, false);
         int mul = getDistributedMultiplier(ses, filters, filter);
 
         return mul * baseCost;

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0c86cdd/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
index f726c04..2f06ac3 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
@@ -29,7 +29,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Cursor;
-import org.h2.engine.Constants;
 import org.h2.engine.Session;
 import org.h2.index.BaseIndex;
 import org.h2.index.Cursor;
@@ -261,7 +260,7 @@ public abstract class GridMergeIndex extends BaseIndex {
 
     /** {@inheritDoc} */
     @Override public double getCost(Session ses, int[] masks, TableFilter[] filters, int filter, SortOrder sortOrder) {
-        return getRowCountApproximation() + Constants.COST_ROW_OFFSET;
+        return getCostRangeIndex(masks, getRowCountApproximation(), filters, filter, sortOrder, true);
     }
 
     /** {@inheritDoc} */


[07/50] [abbrv] ignite git commit: ignite-split2 - renamings

Posted by sb...@apache.org.
ignite-split2 - renamings


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

Branch: refs/heads/ignite-1232
Commit: d86e0aeb154f4152dab95c6ae63ee5661776d2a8
Parents: b3b0312
Author: S.Vladykin <sv...@gridgain.com>
Authored: Sun Dec 6 02:27:29 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Sun Dec 6 02:27:29 2015 +0300

----------------------------------------------------------------------
 .../query/h2/opt/GridH2Collocation.java         | 661 ------------------
 .../query/h2/opt/GridH2CollocationModel.java    | 662 +++++++++++++++++++
 .../query/h2/opt/GridH2IndexBase.java           |   6 +-
 .../query/h2/opt/GridH2QueryContext.java        |  12 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |   2 +-
 5 files changed, 672 insertions(+), 671 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d86e0aeb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java
deleted file mode 100644
index c4cc6a5..0000000
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Collocation.java
+++ /dev/null
@@ -1,661 +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.query.h2.opt;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import org.h2.command.dml.Query;
-import org.h2.command.dml.Select;
-import org.h2.command.dml.SelectUnion;
-import org.h2.expression.Expression;
-import org.h2.expression.ExpressionColumn;
-import org.h2.index.IndexCondition;
-import org.h2.index.ViewIndex;
-import org.h2.table.Column;
-import org.h2.table.SubQueryInfo;
-import org.h2.table.Table;
-import org.h2.table.TableFilter;
-
-/**
- * Collocation model for a query.
- */
-public final class GridH2Collocation {
-    /** */
-    public static final int MULTIPLIER_COLLOCATED = 1;
-
-    /** */
-    private static final int MULTIPLIER_UNICAST = 20;
-
-    /** */
-    private static final int MULTIPLIER_BROADCAST = 80;
-
-    /** */
-    private final GridH2Collocation upper;
-
-    /** */
-    private final int filter;
-
-    /** */
-    private int multiplier;
-
-    /** */
-    private Type type;
-
-    /** */
-    private GridH2Collocation[] children;
-
-    /** */
-    private TableFilter[] childFilters;
-
-    /** */
-    private List<GridH2Collocation> unions;
-
-    /** */
-    private Select select;
-
-    /**
-     * @param upper Upper.
-     * @param filter Filter.
-     */
-    private GridH2Collocation(GridH2Collocation upper, int filter) {
-        this.upper = upper;
-        this.filter = filter;
-    }
-
-    /**
-     * @param upper Upper.
-     * @param filter Filter.
-     * @param unions Unions.
-     * @return Create collocation model.
-     */
-    private static GridH2Collocation createChild(GridH2Collocation upper, int filter, List<GridH2Collocation> unions) {
-        GridH2Collocation child = new GridH2Collocation(upper, filter);
-
-        if (unions != null) {
-            // Bind created child to unions.
-            assert upper == null || upper.child(filter, false) != null;
-
-            unions.add(child);
-
-            child.unions = unions;
-        }
-        else if (upper != null) {
-            // Bind created child to upper model.
-            assert upper.child(filter, false) == null;
-
-            upper.children[filter] = child;
-        }
-
-        return child;
-    }
-
-    /**
-     * @param childFilters New child filters.
-     * @return {@code true} If child filters were updated.
-     */
-    private boolean childFilters(TableFilter[] childFilters) {
-        assert childFilters != null;
-
-        Select select = childFilters[0].getSelect();
-
-        assert this.select == null || this.select == select;
-
-        if (this.select == null) {
-            this.select = select;
-
-            assert this.childFilters == null;
-        }
-        else if (Arrays.equals(this.childFilters, childFilters))
-            return false;
-
-        if (this.childFilters == null) {
-            // We have to clone because H2 reuses array and reorders elements.
-            this.childFilters = childFilters.clone();
-
-            children = new GridH2Collocation[childFilters.length];
-        }
-        else {
-            assert this.childFilters.length == childFilters.length;
-
-            // We have to copy because H2 reuses array and reorders elements.
-            System.arraycopy(childFilters, 0, this.childFilters, 0, childFilters.length);
-
-            Arrays.fill(children, null);
-        }
-
-        // Reset results.
-        type = null;
-        multiplier = 0;
-
-        return true;
-    }
-
-    /**
-     * @param i Index.
-     * @param f Table filter.
-     * @return {@code true} If the child is not a table or view.
-     */
-    private boolean isChildTableOrView(int i, TableFilter f) {
-        if (f == null)
-            f = childFilters[i];
-
-        Table t = f.getTable();
-
-        return t.isView() || t instanceof GridH2Table;
-    }
-
-    /**
-     * Do the needed calculations.
-     */
-    private void calculate() {
-        if (type != null)
-            return;
-
-        if (childFilters != null) {
-            // We are at sub-query.
-            boolean collocated = true;
-            boolean partitioned = false;
-            int maxMultiplier = MULTIPLIER_COLLOCATED;
-
-            for (int i = 0; i < childFilters.length; i++) {
-                GridH2Collocation c = child(i, true);
-
-                Type t = c.type(true);
-
-                if (t.isPartitioned()) {
-                    partitioned = true;
-
-                    if (!t.isCollocated()) {
-                        collocated = false;
-
-                        int m = c.multiplier(true);
-
-                        if (m > maxMultiplier) {
-                            maxMultiplier = m;
-
-                            if (maxMultiplier == MULTIPLIER_BROADCAST)
-                                break;
-                        }
-                    }
-                }
-            }
-
-            type = Type.of(partitioned, collocated);
-            multiplier = maxMultiplier;
-        }
-        else {
-            assert upper != null;
-
-            // We are at table instance.
-            GridH2Table tbl = (GridH2Table)upper.childFilters[filter].getTable();
-
-            // Only partitioned tables will do distributed joins.
-            if (!tbl.isPartitioned()) {
-                type = Type.REPLICATED;
-                multiplier = MULTIPLIER_COLLOCATED;
-
-                return;
-            }
-
-            // If we are the first partitioned table in a join, then we are "base" for all the rest partitioned tables
-            // which will need to get remote result (if there is no affinity condition). Since this query is broadcasted
-            // to all the affinity nodes the "base" does not need to get remote results.
-            if (!upper.findPartitionedTableBefore(filter)) {
-                type = Type.PARTITIONED_COLLOCATED;
-                multiplier = MULTIPLIER_COLLOCATED;
-
-                return;
-            }
-
-            // It is enough to make sure that our previous join by affinity key is collocated, then we are
-            // collocated. If we at least have affinity key condition, then we do unicast which is cheaper.
-            switch (upper.joinedWithCollocated(filter)) {
-                case JOINED_WITH_COLLOCATED:
-                    type = Type.PARTITIONED_COLLOCATED;
-                    multiplier = MULTIPLIER_COLLOCATED;
-
-                    break;
-
-                case HAS_AFFINITY_CONDITION:
-                    type = Type.PARTITIONED_NOT_COLLOCATED;
-                    multiplier = MULTIPLIER_UNICAST;
-
-                    break;
-
-                case NONE:
-                    type = Type.PARTITIONED_NOT_COLLOCATED;
-                    multiplier = MULTIPLIER_BROADCAST;
-
-                    break;
-
-                default:
-                    throw new IllegalStateException();
-            }
-        }
-    }
-
-    /**
-     * @param f Current filter.
-     * @return {@code true} If partitioned table was found.
-     */
-    private boolean findPartitionedTableBefore(int f) {
-        for (int i = 0; i < f; i++) {
-            GridH2Collocation c = child(i, true);
-
-            // The c can be null if it is not a GridH2Table and not a sub-query,
-            // it is a some kind of function table or anything else that considered replicated.
-            if (c != null && c.type(true).isPartitioned())
-                return true;
-        }
-
-        // We have to search globally in upper queries as well.
-        return upper != null && upper.findPartitionedTableBefore(filter);
-    }
-
-    /**
-     * @param f Filter.
-     * @return Affinity join type.
-     */
-    private Affinity joinedWithCollocated(int f) {
-        TableFilter tf = childFilters[f];
-
-        ArrayList<IndexCondition> idxConditions = tf.getIndexConditions();
-
-        int affColId = ((GridH2Table)tf.getTable()).getAffinityKeyColumnId();
-
-        boolean affKeyConditionFound = false;
-
-        for (int i = 0; i < idxConditions.size(); i++) {
-            IndexCondition c = idxConditions.get(i);
-
-            if (c.getCompareType() == IndexCondition.EQUALITY &&
-                c.getColumn().getColumnId() == affColId && c.isEvaluatable()) {
-                affKeyConditionFound = true;
-
-                Expression exp = c.getExpression();
-
-                exp = exp.getNonAliasExpression();
-
-                if (exp instanceof ExpressionColumn) {
-                    ExpressionColumn expCol = (ExpressionColumn)exp;
-
-                    // This is one of our previous joins.
-                    TableFilter prevJoin = expCol.getTableFilter();
-
-                    if (prevJoin != null) {
-                        GridH2Collocation co = child(indexOf(prevJoin), true);
-
-                        if (co != null) {
-                            Type t = co.type(true);
-
-                            if (t.isPartitioned() && t.isCollocated() && isAffinityColumn(prevJoin, expCol))
-                                return Affinity.JOINED_WITH_COLLOCATED;
-                        }
-                    }
-                }
-            }
-        }
-
-        return affKeyConditionFound ? Affinity.HAS_AFFINITY_CONDITION : Affinity.NONE;
-    }
-
-    /**
-     * @param f Table filter.
-     * @return Index.
-     */
-    private int indexOf(TableFilter f) {
-        for (int i = 0; i < childFilters.length; i++) {
-            if (childFilters[i] == f)
-                return i;
-        }
-
-        throw new IllegalStateException();
-    }
-
-    /**
-     * @param f Table filter.
-     * @param expCol Expression column.
-     * @return {@code true} It it is an affinity column.
-     */
-    private static boolean isAffinityColumn(TableFilter f, ExpressionColumn expCol) {
-        Column col = expCol.getColumn();
-
-        if (col == null)
-            return false;
-
-        Table t = col.getTable();
-
-        if (t.isView()) {
-            Query qry = getSubQuery(f);
-
-            return isAffinityColumn(qry, expCol);
-        }
-
-        return t instanceof GridH2Table &&
-            col.getColumnId() == ((GridH2Table)t).getAffinityKeyColumnId();
-    }
-
-    /**
-     * @param qry Query.
-     * @param expCol Expression column.
-     * @return {@code true} It it is an affinity column.
-     */
-    private static boolean isAffinityColumn(Query qry, ExpressionColumn expCol) {
-        if (qry.isUnion()) {
-            SelectUnion union = (SelectUnion)qry;
-
-            return isAffinityColumn(union.getLeft(), expCol) && isAffinityColumn(union.getRight(), expCol);
-        }
-
-        Expression exp = qry.getExpressions().get(expCol.getColumn().getColumnId()).getNonAliasExpression();
-
-        if (exp instanceof ExpressionColumn) {
-            expCol = (ExpressionColumn)exp;
-
-            return isAffinityColumn(expCol.getTableFilter(), expCol);
-        }
-
-        return false;
-    }
-
-    /**
-     * @return Multiplier.
-     */
-    public int calculateMultiplier() {
-        // We don't need multiplier for union here because it will be summarized in H2.
-        return multiplier(false);
-    }
-
-    /**
-     * @param withUnion With respect to union.
-     * @return Multiplier.
-     */
-    private int multiplier(boolean withUnion) {
-        calculate();
-
-        assert multiplier != 0;
-
-        if (withUnion && unions != null) {
-            int maxMultiplier = 0;
-
-            for (int i = 0; i < unions.size(); i++) {
-                int m = unions.get(i).multiplier(false);
-
-                if (m > maxMultiplier)
-                    maxMultiplier = m;
-            }
-
-            return maxMultiplier;
-        }
-
-        return multiplier;
-    }
-
-    /**
-     * @param withUnion With respect to union.
-     * @return Type.
-     */
-    private Type type(boolean withUnion) {
-        calculate();
-
-        assert type != null;
-
-        if (withUnion && unions != null) {
-            Type left = unions.get(0).type(false);
-
-            for (int i = 1; i < unions.size(); i++) {
-                Type right = unions.get(i).type(false);
-
-                if (!left.isCollocated() || !right.isCollocated()) {
-                    left = Type.PARTITIONED_NOT_COLLOCATED;
-
-                    break;
-                }
-                else if (!left.isPartitioned() && !right.isPartitioned())
-                    left = Type.REPLICATED;
-                else
-                    left = Type.PARTITIONED_COLLOCATED;
-            }
-
-            return left;
-        }
-
-        return type;
-    }
-
-    /**
-     * @param i Index.
-     * @param create Create child if needed.
-     * @return Child collocation.
-     */
-    private GridH2Collocation child(int i, boolean create) {
-        GridH2Collocation child = children[i];
-
-        if (child == null && create && isChildTableOrView(i, null)) {
-            TableFilter f = childFilters[i];
-
-            children[i] = child = f.getTable().isView() ?
-                buildCollocationModel(this, i, getSubQuery(f), null) :
-                createChild(this, i, null);
-        }
-
-        return child;
-    }
-
-    /**
-     * @param f Table filter.
-     * @return Sub-query.
-     */
-    private static Query getSubQuery(TableFilter f) {
-        return ((ViewIndex)f.getIndex()).getQuery();
-    }
-
-    /**
-     * @return Unions list.
-     */
-    private List<GridH2Collocation> getOrCreateUnions() {
-        if (unions == null) {
-            unions = new ArrayList<>(4);
-
-            unions.add(this);
-        }
-
-        return unions;
-    }
-
-    /**
-     * @param qctx Query context.
-     * @param info Sub-query info.
-     * @param filters Filters.
-     * @param filter Filter.
-     * @return Collocation.
-     */
-    public static GridH2Collocation buildCollocationModel(GridH2QueryContext qctx, SubQueryInfo info,
-        TableFilter[] filters, int filter) {
-        GridH2Collocation c;
-
-        if (info != null) {
-            // Go up until we reach the root query.
-            c = buildCollocationModel(qctx, info.getUpper(), info.getFilters(), info.getFilter());
-        }
-        else {
-            // We are at the root query.
-            c = qctx.queryCollocation();
-
-            if (c == null) {
-                c = createChild(null, -1, null);
-
-                qctx.queryCollocation(c);
-            }
-        }
-
-        Select select = filters[0].getSelect();
-
-        // Handle union. We have to rely on fact that select will be the same on uppermost select.
-        // For sub-queries we will drop collocation models, so that they will be recalculated anyways.
-        if (c.select != null && c.select != select) {
-            List<GridH2Collocation> unions = c.getOrCreateUnions();
-
-            // Try to find this select in existing unions.
-            // Start with 1 because at 0 it always will be c.
-            for (int i = 1; i < unions.size(); i++) {
-                GridH2Collocation u = unions.get(i);
-
-                if (u.select == select) {
-                    c = u;
-
-                    break;
-                }
-            }
-
-            // Nothing was found, need to create new child in union.
-            if (c.select != select)
-                c = createChild(c.upper, c.filter, unions);
-        }
-
-        c.childFilters(filters);
-
-        return c.child(filter, true);
-    }
-
-    /**
-     * @param qry Query.
-     * @return {@code true} If the query is collocated.
-     */
-    public static boolean isCollocated(Query qry) {
-        return buildCollocationModel(null, -1, qry, null).type(true).isCollocated();
-    }
-
-    /**
-     * @param upper Upper.
-     * @param filter Filter.
-     * @param qry Query.
-     * @param unions Unions.
-     * @return Built model.
-     */
-    private static GridH2Collocation buildCollocationModel(GridH2Collocation upper, int filter, Query qry,
-        List<GridH2Collocation> unions) {
-        if (qry.isUnion()) {
-            if (unions == null)
-                unions = new ArrayList<>();
-
-            SelectUnion union = (SelectUnion)qry;
-
-            GridH2Collocation a = buildCollocationModel(upper, filter, union.getLeft(), unions);
-            GridH2Collocation b = buildCollocationModel(upper, filter, union.getRight(), unions);
-
-            return a == null ? b : a;
-        }
-
-        Select select = (Select)qry;
-
-        List<TableFilter> list = new ArrayList<>();
-
-        for (TableFilter f = select.getTopTableFilter(); f != null; f = f.getJoin())
-            list.add(f);
-
-        TableFilter[] filters = list.toArray(new TableFilter[list.size()]);
-
-        GridH2Collocation c = createChild(upper, filter, unions);
-
-        c.childFilters(filters);
-
-        for (int i = 0; i < filters.length; i++) {
-            TableFilter f = filters[i];
-
-            if (f.getTable().isView())
-                buildCollocationModel(c, i, getSubQuery(f), null);
-            else if (f.getTable() instanceof GridH2Table)
-                createChild(c, i, null);
-        }
-
-        return upper == null ? c : null;
-    }
-
-    /**
-     * Collocation type.
-     */
-    private enum Type {
-        /** */
-        PARTITIONED_COLLOCATED(true, true),
-
-        /** */
-        PARTITIONED_NOT_COLLOCATED(true, false),
-
-        /** */
-        REPLICATED(false, true);
-
-        /** */
-        private final boolean partitioned;
-
-        /** */
-        private final boolean collocated;
-
-        /**
-         * @param partitioned Partitioned.
-         * @param collocated Collocated.
-         */
-        Type(boolean partitioned, boolean collocated) {
-            this.partitioned = partitioned;
-            this.collocated = collocated;
-        }
-
-        /**
-         * @return {@code true} If partitioned.
-         */
-        public boolean isPartitioned() {
-            return partitioned;
-        }
-
-        /**
-         * @return {@code true} If collocated.
-         */
-        public boolean isCollocated() {
-            return collocated;
-        }
-
-        /**
-         * @param partitioned Partitioned.
-         * @param collocated Collocated.
-         * @return Type.
-         */
-        static Type of(boolean partitioned, boolean collocated) {
-            if (collocated)
-                return partitioned ? Type.PARTITIONED_COLLOCATED : Type.REPLICATED;
-
-            assert partitioned;
-
-            return Type.PARTITIONED_NOT_COLLOCATED;
-        }
-    }
-
-    /**
-     * Affinity of a table relative to previous joined tables.
-     */
-    private enum Affinity {
-        /** */
-        NONE,
-
-        /** */
-        HAS_AFFINITY_CONDITION,
-
-        /** */
-        JOINED_WITH_COLLOCATED
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d86e0aeb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2CollocationModel.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2CollocationModel.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2CollocationModel.java
new file mode 100644
index 0000000..2a1437c
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2CollocationModel.java
@@ -0,0 +1,662 @@
+/*
+ * 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.query.h2.opt;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.h2.command.dml.Query;
+import org.h2.command.dml.Select;
+import org.h2.command.dml.SelectUnion;
+import org.h2.expression.Expression;
+import org.h2.expression.ExpressionColumn;
+import org.h2.index.IndexCondition;
+import org.h2.index.ViewIndex;
+import org.h2.table.Column;
+import org.h2.table.SubQueryInfo;
+import org.h2.table.Table;
+import org.h2.table.TableFilter;
+
+/**
+ * Collocation model for a query.
+ */
+public final class GridH2CollocationModel {
+    /** */
+    public static final int MULTIPLIER_COLLOCATED = 1;
+
+    /** */
+    private static final int MULTIPLIER_UNICAST = 20;
+
+    /** */
+    private static final int MULTIPLIER_BROADCAST = 80;
+
+    /** */
+    private final GridH2CollocationModel upper;
+
+    /** */
+    private final int filter;
+
+    /** */
+    private int multiplier;
+
+    /** */
+    private Type type;
+
+    /** */
+    private GridH2CollocationModel[] children;
+
+    /** */
+    private TableFilter[] childFilters;
+
+    /** */
+    private List<GridH2CollocationModel> unions;
+
+    /** */
+    private Select select;
+
+    /**
+     * @param upper Upper.
+     * @param filter Filter.
+     */
+    private GridH2CollocationModel(GridH2CollocationModel upper, int filter) {
+        this.upper = upper;
+        this.filter = filter;
+    }
+
+    /**
+     * @param upper Upper.
+     * @param filter Filter.
+     * @param unions Unions.
+     * @return Created child collocation model.
+     */
+    private static GridH2CollocationModel createChildModel(GridH2CollocationModel upper, int filter,
+        List<GridH2CollocationModel> unions) {
+        GridH2CollocationModel child = new GridH2CollocationModel(upper, filter);
+
+        if (unions != null) {
+            // Bind created child to unions.
+            assert upper == null || upper.child(filter, false) != null;
+
+            unions.add(child);
+
+            child.unions = unions;
+        }
+        else if (upper != null) {
+            // Bind created child to upper model.
+            assert upper.child(filter, false) == null;
+
+            upper.children[filter] = child;
+        }
+
+        return child;
+    }
+
+    /**
+     * @param childFilters New child filters.
+     * @return {@code true} If child filters were updated.
+     */
+    private boolean childFilters(TableFilter[] childFilters) {
+        assert childFilters != null;
+
+        Select select = childFilters[0].getSelect();
+
+        assert this.select == null || this.select == select;
+
+        if (this.select == null) {
+            this.select = select;
+
+            assert this.childFilters == null;
+        }
+        else if (Arrays.equals(this.childFilters, childFilters))
+            return false;
+
+        if (this.childFilters == null) {
+            // We have to clone because H2 reuses array and reorders elements.
+            this.childFilters = childFilters.clone();
+
+            children = new GridH2CollocationModel[childFilters.length];
+        }
+        else {
+            assert this.childFilters.length == childFilters.length;
+
+            // We have to copy because H2 reuses array and reorders elements.
+            System.arraycopy(childFilters, 0, this.childFilters, 0, childFilters.length);
+
+            Arrays.fill(children, null);
+        }
+
+        // Reset results.
+        type = null;
+        multiplier = 0;
+
+        return true;
+    }
+
+    /**
+     * @param i Index.
+     * @param f Table filter.
+     * @return {@code true} If the child is not a table or view.
+     */
+    private boolean isChildTableOrView(int i, TableFilter f) {
+        if (f == null)
+            f = childFilters[i];
+
+        Table t = f.getTable();
+
+        return t.isView() || t instanceof GridH2Table;
+    }
+
+    /**
+     * Do the needed calculations.
+     */
+    private void calculate() {
+        if (type != null)
+            return;
+
+        if (childFilters != null) {
+            // We are at sub-query.
+            boolean collocated = true;
+            boolean partitioned = false;
+            int maxMultiplier = MULTIPLIER_COLLOCATED;
+
+            for (int i = 0; i < childFilters.length; i++) {
+                GridH2CollocationModel child = child(i, true);
+
+                Type t = child.type(true);
+
+                if (t.isPartitioned()) {
+                    partitioned = true;
+
+                    if (!t.isCollocated()) {
+                        collocated = false;
+
+                        int m = child.multiplier(true);
+
+                        if (m > maxMultiplier) {
+                            maxMultiplier = m;
+
+                            if (maxMultiplier == MULTIPLIER_BROADCAST)
+                                break;
+                        }
+                    }
+                }
+            }
+
+            type = Type.of(partitioned, collocated);
+            multiplier = maxMultiplier;
+        }
+        else {
+            assert upper != null;
+
+            // We are at table instance.
+            GridH2Table tbl = (GridH2Table)upper.childFilters[filter].getTable();
+
+            // Only partitioned tables will do distributed joins.
+            if (!tbl.isPartitioned()) {
+                type = Type.REPLICATED;
+                multiplier = MULTIPLIER_COLLOCATED;
+
+                return;
+            }
+
+            // If we are the first partitioned table in a join, then we are "base" for all the rest partitioned tables
+            // which will need to get remote result (if there is no affinity condition). Since this query is broadcasted
+            // to all the affinity nodes the "base" does not need to get remote results.
+            if (!upper.findPartitionedTableBefore(filter)) {
+                type = Type.PARTITIONED_COLLOCATED;
+                multiplier = MULTIPLIER_COLLOCATED;
+
+                return;
+            }
+
+            // It is enough to make sure that our previous join by affinity key is collocated, then we are
+            // collocated. If we at least have affinity key condition, then we do unicast which is cheaper.
+            switch (upper.joinedWithCollocated(filter)) {
+                case JOINED_WITH_COLLOCATED:
+                    type = Type.PARTITIONED_COLLOCATED;
+                    multiplier = MULTIPLIER_COLLOCATED;
+
+                    break;
+
+                case HAS_AFFINITY_CONDITION:
+                    type = Type.PARTITIONED_NOT_COLLOCATED;
+                    multiplier = MULTIPLIER_UNICAST;
+
+                    break;
+
+                case NONE:
+                    type = Type.PARTITIONED_NOT_COLLOCATED;
+                    multiplier = MULTIPLIER_BROADCAST;
+
+                    break;
+
+                default:
+                    throw new IllegalStateException();
+            }
+        }
+    }
+
+    /**
+     * @param f Current filter.
+     * @return {@code true} If partitioned table was found.
+     */
+    private boolean findPartitionedTableBefore(int f) {
+        for (int i = 0; i < f; i++) {
+            GridH2CollocationModel child = child(i, true);
+
+            // The c can be null if it is not a GridH2Table and not a sub-query,
+            // it is a some kind of function table or anything else that considered replicated.
+            if (child != null && child.type(true).isPartitioned())
+                return true;
+        }
+
+        // We have to search globally in upper queries as well.
+        return upper != null && upper.findPartitionedTableBefore(filter);
+    }
+
+    /**
+     * @param f Filter.
+     * @return Affinity join type.
+     */
+    private Affinity joinedWithCollocated(int f) {
+        TableFilter tf = childFilters[f];
+
+        ArrayList<IndexCondition> idxConditions = tf.getIndexConditions();
+
+        int affColId = ((GridH2Table)tf.getTable()).getAffinityKeyColumnId();
+
+        boolean affKeyConditionFound = false;
+
+        for (int i = 0; i < idxConditions.size(); i++) {
+            IndexCondition c = idxConditions.get(i);
+
+            if (c.getCompareType() == IndexCondition.EQUALITY &&
+                c.getColumn().getColumnId() == affColId && c.isEvaluatable()) {
+                affKeyConditionFound = true;
+
+                Expression exp = c.getExpression();
+
+                exp = exp.getNonAliasExpression();
+
+                if (exp instanceof ExpressionColumn) {
+                    ExpressionColumn expCol = (ExpressionColumn)exp;
+
+                    // This is one of our previous joins.
+                    TableFilter prevJoin = expCol.getTableFilter();
+
+                    if (prevJoin != null) {
+                        GridH2CollocationModel cm = child(indexOf(prevJoin), true);
+
+                        if (cm != null) {
+                            Type t = cm.type(true);
+
+                            if (t.isPartitioned() && t.isCollocated() && isAffinityColumn(prevJoin, expCol))
+                                return Affinity.JOINED_WITH_COLLOCATED;
+                        }
+                    }
+                }
+            }
+        }
+
+        return affKeyConditionFound ? Affinity.HAS_AFFINITY_CONDITION : Affinity.NONE;
+    }
+
+    /**
+     * @param f Table filter.
+     * @return Index.
+     */
+    private int indexOf(TableFilter f) {
+        for (int i = 0; i < childFilters.length; i++) {
+            if (childFilters[i] == f)
+                return i;
+        }
+
+        throw new IllegalStateException();
+    }
+
+    /**
+     * @param f Table filter.
+     * @param expCol Expression column.
+     * @return {@code true} It it is an affinity column.
+     */
+    private static boolean isAffinityColumn(TableFilter f, ExpressionColumn expCol) {
+        Column col = expCol.getColumn();
+
+        if (col == null)
+            return false;
+
+        Table t = col.getTable();
+
+        if (t.isView()) {
+            Query qry = getSubQuery(f);
+
+            return isAffinityColumn(qry, expCol);
+        }
+
+        return t instanceof GridH2Table &&
+            col.getColumnId() == ((GridH2Table)t).getAffinityKeyColumnId();
+    }
+
+    /**
+     * @param qry Query.
+     * @param expCol Expression column.
+     * @return {@code true} It it is an affinity column.
+     */
+    private static boolean isAffinityColumn(Query qry, ExpressionColumn expCol) {
+        if (qry.isUnion()) {
+            SelectUnion union = (SelectUnion)qry;
+
+            return isAffinityColumn(union.getLeft(), expCol) && isAffinityColumn(union.getRight(), expCol);
+        }
+
+        Expression exp = qry.getExpressions().get(expCol.getColumn().getColumnId()).getNonAliasExpression();
+
+        if (exp instanceof ExpressionColumn) {
+            expCol = (ExpressionColumn)exp;
+
+            return isAffinityColumn(expCol.getTableFilter(), expCol);
+        }
+
+        return false;
+    }
+
+    /**
+     * @return Multiplier.
+     */
+    public int calculateMultiplier() {
+        // We don't need multiplier for union here because it will be summarized in H2.
+        return multiplier(false);
+    }
+
+    /**
+     * @param withUnion With respect to union.
+     * @return Multiplier.
+     */
+    private int multiplier(boolean withUnion) {
+        calculate();
+
+        assert multiplier != 0;
+
+        if (withUnion && unions != null) {
+            int maxMultiplier = 0;
+
+            for (int i = 0; i < unions.size(); i++) {
+                int m = unions.get(i).multiplier(false);
+
+                if (m > maxMultiplier)
+                    maxMultiplier = m;
+            }
+
+            return maxMultiplier;
+        }
+
+        return multiplier;
+    }
+
+    /**
+     * @param withUnion With respect to union.
+     * @return Type.
+     */
+    private Type type(boolean withUnion) {
+        calculate();
+
+        assert type != null;
+
+        if (withUnion && unions != null) {
+            Type left = unions.get(0).type(false);
+
+            for (int i = 1; i < unions.size(); i++) {
+                Type right = unions.get(i).type(false);
+
+                if (!left.isCollocated() || !right.isCollocated()) {
+                    left = Type.PARTITIONED_NOT_COLLOCATED;
+
+                    break;
+                }
+                else if (!left.isPartitioned() && !right.isPartitioned())
+                    left = Type.REPLICATED;
+                else
+                    left = Type.PARTITIONED_COLLOCATED;
+            }
+
+            return left;
+        }
+
+        return type;
+    }
+
+    /**
+     * @param i Index.
+     * @param create Create child if needed.
+     * @return Child collocation.
+     */
+    private GridH2CollocationModel child(int i, boolean create) {
+        GridH2CollocationModel child = children[i];
+
+        if (child == null && create && isChildTableOrView(i, null)) {
+            TableFilter f = childFilters[i];
+
+            children[i] = child = f.getTable().isView() ?
+                buildCollocationModel(this, i, getSubQuery(f), null) :
+                createChildModel(this, i, null);
+        }
+
+        return child;
+    }
+
+    /**
+     * @param f Table filter.
+     * @return Sub-query.
+     */
+    private static Query getSubQuery(TableFilter f) {
+        return ((ViewIndex)f.getIndex()).getQuery();
+    }
+
+    /**
+     * @return Unions list.
+     */
+    private List<GridH2CollocationModel> getOrCreateUnions() {
+        if (unions == null) {
+            unions = new ArrayList<>(4);
+
+            unions.add(this);
+        }
+
+        return unions;
+    }
+
+    /**
+     * @param qctx Query context.
+     * @param info Sub-query info.
+     * @param filters Filters.
+     * @param filter Filter.
+     * @return Collocation.
+     */
+    public static GridH2CollocationModel buildCollocationModel(GridH2QueryContext qctx, SubQueryInfo info,
+        TableFilter[] filters, int filter) {
+        GridH2CollocationModel cm;
+
+        if (info != null) {
+            // Go up until we reach the root query.
+            cm = buildCollocationModel(qctx, info.getUpper(), info.getFilters(), info.getFilter());
+        }
+        else {
+            // We are at the root query.
+            cm = qctx.queryCollocationModel();
+
+            if (cm == null) {
+                cm = createChildModel(null, -1, null);
+
+                qctx.queryCollocationModel(cm);
+            }
+        }
+
+        Select select = filters[0].getSelect();
+
+        // Handle union. We have to rely on fact that select will be the same on uppermost select.
+        // For sub-queries we will drop collocation models, so that they will be recalculated anyways.
+        if (cm.select != null && cm.select != select) {
+            List<GridH2CollocationModel> unions = cm.getOrCreateUnions();
+
+            // Try to find this select in existing unions.
+            // Start with 1 because at 0 it always will be c.
+            for (int i = 1; i < unions.size(); i++) {
+                GridH2CollocationModel u = unions.get(i);
+
+                if (u.select == select) {
+                    cm = u;
+
+                    break;
+                }
+            }
+
+            // Nothing was found, need to create new child in union.
+            if (cm.select != select)
+                cm = createChildModel(cm.upper, cm.filter, unions);
+        }
+
+        cm.childFilters(filters);
+
+        return cm.child(filter, true);
+    }
+
+    /**
+     * @param qry Query.
+     * @return {@code true} If the query is collocated.
+     */
+    public static boolean isCollocated(Query qry) {
+        return buildCollocationModel(null, -1, qry, null).type(true).isCollocated();
+    }
+
+    /**
+     * @param upper Upper.
+     * @param filter Filter.
+     * @param qry Query.
+     * @param unions Unions.
+     * @return Built model.
+     */
+    private static GridH2CollocationModel buildCollocationModel(GridH2CollocationModel upper, int filter, Query qry,
+        List<GridH2CollocationModel> unions) {
+        if (qry.isUnion()) {
+            if (unions == null)
+                unions = new ArrayList<>();
+
+            SelectUnion union = (SelectUnion)qry;
+
+            GridH2CollocationModel a = buildCollocationModel(upper, filter, union.getLeft(), unions);
+            GridH2CollocationModel b = buildCollocationModel(upper, filter, union.getRight(), unions);
+
+            return a == null ? b : a;
+        }
+
+        Select select = (Select)qry;
+
+        List<TableFilter> list = new ArrayList<>();
+
+        for (TableFilter f = select.getTopTableFilter(); f != null; f = f.getJoin())
+            list.add(f);
+
+        TableFilter[] filters = list.toArray(new TableFilter[list.size()]);
+
+        GridH2CollocationModel cm = createChildModel(upper, filter, unions);
+
+        cm.childFilters(filters);
+
+        for (int i = 0; i < filters.length; i++) {
+            TableFilter f = filters[i];
+
+            if (f.getTable().isView())
+                buildCollocationModel(cm, i, getSubQuery(f), null);
+            else if (f.getTable() instanceof GridH2Table)
+                createChildModel(cm, i, null);
+        }
+
+        return upper == null ? cm : null;
+    }
+
+    /**
+     * Collocation type.
+     */
+    private enum Type {
+        /** */
+        PARTITIONED_COLLOCATED(true, true),
+
+        /** */
+        PARTITIONED_NOT_COLLOCATED(true, false),
+
+        /** */
+        REPLICATED(false, true);
+
+        /** */
+        private final boolean partitioned;
+
+        /** */
+        private final boolean collocated;
+
+        /**
+         * @param partitioned Partitioned.
+         * @param collocated Collocated.
+         */
+        Type(boolean partitioned, boolean collocated) {
+            this.partitioned = partitioned;
+            this.collocated = collocated;
+        }
+
+        /**
+         * @return {@code true} If partitioned.
+         */
+        public boolean isPartitioned() {
+            return partitioned;
+        }
+
+        /**
+         * @return {@code true} If collocated.
+         */
+        public boolean isCollocated() {
+            return collocated;
+        }
+
+        /**
+         * @param partitioned Partitioned.
+         * @param collocated Collocated.
+         * @return Type.
+         */
+        static Type of(boolean partitioned, boolean collocated) {
+            if (collocated)
+                return partitioned ? Type.PARTITIONED_COLLOCATED : Type.REPLICATED;
+
+            assert partitioned;
+
+            return Type.PARTITIONED_NOT_COLLOCATED;
+        }
+    }
+
+    /**
+     * Affinity of a table relative to previous joined tables.
+     */
+    private enum Affinity {
+        /** */
+        NONE,
+
+        /** */
+        HAS_AFFINITY_CONDITION,
+
+        /** */
+        JOINED_WITH_COLLOCATED
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d86e0aeb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
index b063528..20795bc 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2IndexBase.java
@@ -36,7 +36,7 @@ import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow.KEY_COL;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2AbstractKeyValueRow.VAL_COL;
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2Collocation.buildCollocationModel;
+import static org.apache.ignite.internal.processors.query.h2.opt.GridH2CollocationModel.buildCollocationModel;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.PREPARE;
 
 /**
@@ -140,7 +140,7 @@ public abstract class GridH2IndexBase extends BaseIndex {
         // and thus multiplier will be always the same, so it will not affect choice of index.
         // Query expressions can not be distributed as well.
         if (qctx == null || qctx.type() != PREPARE || !qctx.distributedJoins() || ses.isPreparingQueryExpression())
-            return GridH2Collocation.MULTIPLIER_COLLOCATED;
+            return GridH2CollocationModel.MULTIPLIER_COLLOCATED;
 
         // We have to clear this cache because normally sub-query plan cost does not depend on anything
         // other than index condition masks and sort order, but in our case it can depend on order
@@ -149,7 +149,7 @@ public abstract class GridH2IndexBase extends BaseIndex {
 
         assert filters != null;
 
-        GridH2Collocation c = buildCollocationModel(qctx, ses.getSubQueryInfo(), filters, filter);
+        GridH2CollocationModel c = buildCollocationModel(qctx, ses.getSubQueryInfo(), filters, filter);
 
         return c.calculateMultiplier();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d86e0aeb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
index 29bbbaf..8812086 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2QueryContext.java
@@ -79,7 +79,7 @@ public class GridH2QueryContext {
     private int pageSize;
 
     /** */
-    private GridH2Collocation qryCollocation;
+    private GridH2CollocationModel qryCollocationModel;
 
     /**
      * @param locNodeId Local node ID.
@@ -115,15 +115,15 @@ public class GridH2QueryContext {
     /**
      * @return Query collocation model.
      */
-    public GridH2Collocation queryCollocation() {
-        return qryCollocation;
+    public GridH2CollocationModel queryCollocationModel() {
+        return qryCollocationModel;
     }
 
     /**
-     * @param qryCollocation Query collocation model.
+     * @param qryCollocationModel Query collocation model.
      */
-    public void queryCollocation(GridH2Collocation qryCollocation) {
-        this.qryCollocation = qryCollocation;
+    public void queryCollocationModel(GridH2CollocationModel qryCollocationModel) {
+        this.qryCollocationModel = qryCollocationModel;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/d86e0aeb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
index 034eebe..0d41e95 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
@@ -37,7 +37,7 @@ import org.h2.table.IndexColumn;
 import org.h2.util.IntArray;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2Collocation.isCollocated;
+import static org.apache.ignite.internal.processors.query.h2.opt.GridH2CollocationModel.isCollocated;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.AVG;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.CAST;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlFunctionType.COUNT;


[50/50] [abbrv] ignite git commit: ignite-split2 - H2 version

Posted by sb...@apache.org.
ignite-split2 - H2 version


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

Branch: refs/heads/ignite-1232
Commit: 76d5635f33c114fb127ee38c8950dffe58a7556f
Parents: d9d6656
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Feb 15 10:29:01 2016 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Feb 15 10:29:01 2016 +0300

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


http://git-wip-us.apache.org/repos/asf/ignite/blob/76d5635f/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 608f648..b91cf0b 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -70,7 +70,7 @@
         <guava.version>18.0</guava.version>
         <guava14.version>14.0.1</guava14.version>
         <guava16.version>16.0.1</guava16.version>
-        <h2.version>1.0-SNAPSHOT</h2.version>
+        <h2.version>1.4.191</h2.version>
         <hadoop.version>2.4.1</hadoop.version>
         <httpclient.version>4.5.1</httpclient.version>
         <httpcore.version>4.4.3</httpcore.version>


[25/50] [abbrv] ignite git commit: ignite-split2 - renaming

Posted by sb...@apache.org.
ignite-split2 - renaming


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

Branch: refs/heads/ignite-1232
Commit: 6a9a61aedbf49a360a5e777f7728bd76f34ac36c
Parents: 1df1162
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Dec 15 02:15:09 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Dec 15 02:15:09 2015 +0300

----------------------------------------------------------------------
 .../processors/query/h2/opt/GridH2TreeIndex.java        | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6a9a61ae/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
index 9e5acf6..e21e6af 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2TreeIndex.java
@@ -936,7 +936,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
     /**
      * Simple cursor from a single node.
      */
-    private static class SimpleCursor implements Cursor {
+    private static class UnicastCursor implements Cursor {
         /** */
         final int rangeId;
 
@@ -948,7 +948,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
          * @param nodes Remote nodes.
          * @param rangeStreams Range streams.
          */
-        private SimpleCursor(int rangeId, Collection<ClusterNode> nodes, Map<ClusterNode,RangeStream> rangeStreams) {
+        private UnicastCursor(int rangeId, Collection<ClusterNode> nodes, Map<ClusterNode,RangeStream> rangeStreams) {
             assert nodes.size() == 1;
 
             this.rangeId = rangeId;
@@ -981,7 +981,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
     /**
      * Merge cursor from multiple nodes.
      */
-    private class MergeCursor implements Cursor, Comparator<RangeStream> {
+    private class BroadcastCursor implements Cursor, Comparator<RangeStream> {
         /** */
         final int rangeId;
 
@@ -999,7 +999,7 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
          * @param nodes Remote nodes.
          * @param rangeStreams Range streams.
          */
-        private MergeCursor(int rangeId, Collection<ClusterNode> nodes, Map<ClusterNode,RangeStream> rangeStreams) {
+        private BroadcastCursor(int rangeId, Collection<ClusterNode> nodes, Map<ClusterNode,RangeStream> rangeStreams) {
             assert nodes.size() > 1;
 
             this.rangeId = rangeId;
@@ -1275,8 +1275,8 @@ public class GridH2TreeIndex extends GridH2IndexBase implements Comparator<GridS
             }
 
             fut = new DoneFuture<>(nodes.size() == 1 ?
-                new SimpleCursor(rangeId, nodes, rangeStreams) :
-                new MergeCursor(rangeId, nodes, rangeStreams));
+                new UnicastCursor(rangeId, nodes, rangeStreams) :
+                new BroadcastCursor(rangeId, nodes, rangeStreams));
 
             res.add(fut);
 


[13/50] [abbrv] ignite git commit: ignite-split2 - minor

Posted by sb...@apache.org.
ignite-split2 - minor


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

Branch: refs/heads/ignite-1232
Commit: f356f4f1914fad1a26d01a7ec59591289e2a45b3
Parents: d87978b
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Dec 7 09:02:39 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Dec 7 09:02:39 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/query/IgniteSqlSplitterSelfTest.java     | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f356f4f1/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
index 57df338..879f343 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlSplitterSelfTest.java
@@ -324,7 +324,7 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
         }
     }
 
-    private static class Person {
+    private static class Person implements Serializable {
         @QuerySqlField
         int orgId;
 
@@ -332,7 +332,7 @@ public class IgniteSqlSplitterSelfTest extends GridCommonAbstractTest {
         String name;
     }
 
-    private static class Organization {
+    private static class Organization implements Serializable {
         @QuerySqlField
         String name;
     }