You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2016/11/17 10:51:24 UTC

[01/50] [abbrv] ignite git commit: Merge branch 'ignite-1.6.10' into ignite-1.7.3

Repository: ignite
Updated Branches:
  refs/heads/ignite-2693 7fd543c34 -> c5390868f


Merge branch 'ignite-1.6.10' into ignite-1.7.3


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

Branch: refs/heads/ignite-2693
Commit: 5fac786b6dbb179127ac725180acb54d0f6f4b0a
Parents: d0f4b23 a863eee
Author: devozerov <vo...@gridgain.com>
Authored: Mon Oct 31 21:31:05 2016 +0300
Committer: thatcoach <pp...@list.ru>
Committed: Mon Oct 31 21:58:20 2016 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteScheduler.java |  13 +
 .../cache/query/QueryCancelledException.java    |  35 +++
 .../apache/ignite/cache/query/QueryCursor.java  |   8 +-
 .../ignite/cache/query/SqlFieldsQuery.java      |  26 ++
 .../org/apache/ignite/cache/query/SqlQuery.java |  25 ++
 .../ignite/internal/IgniteSchedulerImpl.java    |  18 ++
 .../processors/cache/QueryCursorImpl.java       |  92 +++++--
 .../closure/GridClosureProcessor.java           |   1 +
 .../processors/query/GridQueryCancel.java       |  84 +++++++
 .../processors/query/GridQueryFieldsResult.java |   3 +-
 .../query/GridQueryFieldsResultAdapter.java     |   3 +-
 .../processors/query/GridQueryIndexing.java     |  11 +-
 .../processors/query/GridQueryProcessor.java    | 105 ++++++--
 .../twostep/messages/GridQueryFailResponse.java |  34 ++-
 .../h2/twostep/messages/GridQueryRequest.java   |  31 ++-
 .../junits/GridTestKernalContext.java           |   1 -
 .../processors/query/h2/IgniteH2Indexing.java   | 160 +++++++++---
 .../query/h2/twostep/GridMapQueryExecutor.java  |  66 +++--
 .../h2/twostep/GridReduceQueryExecutor.java     | 117 ++++++---
 ...niteCacheDistributedQueryCancelSelfTest.java | 176 +++++++++++++
 ...butedQueryStopOnCancelOrTimeoutSelfTest.java | 248 +++++++++++++++++++
 ...cheQueryAbstractDistributedJoinSelfTest.java |   7 +
 .../IgniteCacheQueryNodeRestartSelfTest2.java   | 125 ++++++----
 ...nCancelOrTimeoutDistributedJoinSelfTest.java |   7 +
 ...eCacheLocalQueryCancelOrTimeoutSelfTest.java | 158 ++++++++++++
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   4 +-
 .../IgniteCacheQuerySelfTestSuite2.java         |   8 +
 27 files changed, 1388 insertions(+), 178 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5fac786b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
index 48dab6b,d1a5117..d3f85af
--- 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
@@@ -56,12 -58,9 +58,15 @@@ public final class SqlFieldsQuery exten
      /** Collocation flag. */
      private boolean collocated;
  
+     /** Query timeout in millis. */
+     private int timeout;
+ 
 +    /** */
 +    private boolean enforceJoinOrder;
 +
 +    /** */
 +    private boolean distributedJoins;
 +
      /**
       * Constructs SQL fields query.
       *

http://git-wip-us.apache.org/repos/asf/ignite/blob/5fac786b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/cache/query/SqlQuery.java
index f809b8d,51c6cb5..83e171d
--- 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,9 -44,9 +44,12 @@@ public final class SqlQuery<K, V> exten
      @GridToStringInclude
      private Object[] args;
  
+     /** Timeout in millis. */
+     private int timeout;
+ 
 +    /** */
 +    private boolean distributedJoins;
 +
      /**
       * Constructs query for the given type name and SQL query.
       *

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/5fac786b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index 643cb8c,b1b3c68..6bffa5d
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@@ -80,13 -80,15 +80,15 @@@ public interface GridQueryIndexing 
       * @param spaceName Space name.
       * @param qry Query.
       * @param params Query parameters.
 -     * @param filters Space name and key filters.
 +     * @param filter Space name and key filter.
 +     * @param enforceJoinOrder Enforce join order of tables in the query.
+      * @param timeout Query timeout in milliseconds.
+      * @param cancel Query cancel.
       * @return Query result.
       * @throws IgniteCheckedException If failed.
       */
 -    public GridQueryFieldsResult execute(@Nullable String spaceName, String qry,
 -        Collection<Object> params, IndexingQueryFilter filters, int timeout, GridQueryCancel cancel)
 -        throws IgniteCheckedException;
 +    public GridQueryFieldsResult queryLocalSqlFields(@Nullable String spaceName, String qry,
-         Collection<Object> params, IndexingQueryFilter filter, boolean enforceJoinOrder) throws IgniteCheckedException;
++        Collection<Object> params, IndexingQueryFilter filter, boolean enforceJoinOrder, int timeout, GridQueryCancel cancel) throws IgniteCheckedException;
  
      /**
       * Executes regular query.

http://git-wip-us.apache.org/repos/asf/ignite/blob/5fac786b/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 12cf962,3d185c6..27c0b71
--- 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
@@@ -17,6 -17,6 +17,7 @@@
  
  package org.apache.ignite.internal.processors.query;
  
++import java.util.concurrent.TimeUnit;
  import java.lang.reflect.AccessibleObject;
  import java.lang.reflect.Field;
  import java.lang.reflect.Member;
@@@ -719,6 -740,43 +734,43 @@@ public class GridQueryProcessor extend
      }
  
      /**
+      * @param space Space.
+      * @param clause Clause.
+      * @param params Parameters collection.
+      * @param resType Result type.
+      * @param filters Filters.
+      * @return Key/value rows.
+      * @throws IgniteCheckedException If failed.
+      */
+     @SuppressWarnings("unchecked")
+     public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> query(final String space, final String clause,
+         final Collection<Object> params, final String resType, final IndexingQueryFilter filters)
+         throws IgniteCheckedException {
+         checkEnabled();
+ 
+         if (!busyLock.enterBusy())
+             throw new IllegalStateException("Failed to execute query (grid is stopping).");
+ 
+         try {
+             final GridCacheContext<?, ?> cctx = ctx.cache().internalCache(space).context();
+ 
+             return executeQuery(cctx, new IgniteOutClosureX<GridCloseableIterator<IgniteBiTuple<K, V>>>() {
+                 @Override public GridCloseableIterator<IgniteBiTuple<K, V>> applyx() throws IgniteCheckedException {
+                     TypeDescriptor type = typesByName.get(new TypeName(space, resType));
+ 
+                     if (type == null || !type.registered())
+                         throw new CacheException("Failed to find SQL table for type: " + resType);
+ 
 -                    return idx.query(space, clause, params, type, filters);
++                    return idx.queryLocalSql(space, clause, params, type, filters);
+                 }
+             }, false);
+         }
+         finally {
+             busyLock.leaveBusy();
+         }
+     }
+ 
+     /**
       * @param cctx Cache context.
       * @param qry Query.
       * @return Cursor.
@@@ -868,6 -925,24 +920,24 @@@
      }
  
      /**
+      * @param timeout Timeout.
+      * @param timeUnit Time unit.
+      * @return Converted time.
+      */
+     public static int validateTimeout(int timeout, TimeUnit timeUnit) {
+         A.ensure(timeUnit != TimeUnit.MICROSECONDS && timeUnit != TimeUnit.NANOSECONDS,
 -                "timeUnit minimal resolution is millisecond.");
++            "timeUnit minimal resolution is millisecond.");
+ 
+         A.ensure(timeout >= 0, "timeout value should be non-negative.");
+ 
+         long tmp = TimeUnit.MILLISECONDS.convert(timeout, timeUnit);
+ 
+         A.ensure(timeout <= Integer.MAX_VALUE, "timeout value too large.");
+ 
+         return (int) tmp;
+     }
+ 
+     /**
       * Closeable iterator.
       */
      private interface ClIter<X> extends AutoCloseable, Iterator<X> {
@@@ -888,14 -963,13 +958,13 @@@
  
              return executeQuery(cctx, new IgniteOutClosureX<QueryCursor<List<?>>>() {
                  @Override public QueryCursor<List<?>> applyx() throws IgniteCheckedException {
-                     String space = cctx.name();
-                     String sql = qry.getSql();
-                     Object[] args = qry.getArgs();
+                     final String space = cctx.name();
+                     final String sql = qry.getSql();
+                     final Object[] args = qry.getArgs();
+                     final GridQueryCancel cancel = new GridQueryCancel();
  
 -                    final GridQueryFieldsResult res = idx.execute(space, sql, F.asList(args),
 -                        idx.backupFilter(null, requestTopVer.get(), null), qry.getTimeout(), cancel);
 +                    final GridQueryFieldsResult res = idx.queryLocalSqlFields(space, sql, F.asList(args),
-                         idx.backupFilter(requestTopVer.get(), null), qry.isEnforceJoinOrder());
- 
-                     sendQueryExecutedEvent(sql, args);
++                        idx.backupFilter(requestTopVer.get(), null), qry.isEnforceJoinOrder(), qry.getTimeout(), cancel);
  
                      QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(new Iterable<List<?>>() {
                          @Override public Iterator<List<?>> iterator() {
@@@ -2412,4 -2501,4 +2487,4 @@@
      private enum IndexType {
          ASC, DESC, TEXT
      }
--}
++}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5fac786b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java
index f7de86c,550cf9b..6e42f1c
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java
@@@ -167,9 -169,16 +173,16 @@@ public class GridQueryRequest implement
      }
  
      /**
+      * @return Timeout.
+      */
+     public int timeout() {
+         return this.timeout;
+     }
+ 
+     /**
       * @return Queries.
       */
 -    public Collection<GridCacheSqlQuery> queries() throws IgniteCheckedException {
 +    public Collection<GridCacheSqlQuery> queries() {
          return qrys;
      }
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/5fac786b/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 331b6f9,ab332c1..ed42bc6
--- 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
@@@ -75,8 -76,8 +77,9 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.cache.GridCacheContext;
  import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
  import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 +import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable;
  import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
+ import org.apache.ignite.internal.processors.query.GridQueryCancel;
  import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
  import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
  import org.apache.ignite.internal.processors.query.GridQueryFieldsResultAdapter;
@@@ -132,9 -128,9 +135,10 @@@ import org.h2.index.Index
  import org.h2.index.SpatialIndex;
  import org.h2.jdbc.JdbcConnection;
  import org.h2.jdbc.JdbcPreparedStatement;
+ import org.h2.jdbc.JdbcStatement;
  import org.h2.message.DbException;
  import org.h2.mvstore.cache.CacheLongKeyLIRS;
 +import org.h2.result.SortOrder;
  import org.h2.server.web.WebServer;
  import org.h2.table.Column;
  import org.h2.table.IndexColumn;
@@@ -390,7 -356,7 +394,7 @@@ public class IgniteH2Indexing implement
  
              PreparedStatement stmt = cache.get(sql);
  
-             if (stmt != null && !stmt.isClosed()) {
 -            if (stmt != null && !((JdbcStatement)stmt).wasCancelled()) {
++            if (stmt != null && !stmt.isClosed() && !((JdbcStatement)stmt).wasCancelled()) {
                  assert stmt.getConnection() == c;
  
                  return stmt;
@@@ -764,31 -733,36 +768,36 @@@
  
      /** {@inheritDoc} */
      @SuppressWarnings("unchecked")
 -    @Override public GridQueryFieldsResult execute(@Nullable final String spaceName, final String qry,
 -        @Nullable final Collection<Object> params, final IndexingQueryFilter filters,
 +    @Override public GridQueryFieldsResult queryLocalSqlFields(@Nullable final String spaceName, final String qry,
-         @Nullable final Collection<Object> params, final IndexingQueryFilter filters, boolean enforceJoinOrder)
++        @Nullable final Collection<Object> params, final IndexingQueryFilter filters, boolean enforceJoinOrder,
+         final int timeout, final GridQueryCancel cancel)
          throws IgniteCheckedException {
-         Connection conn = connectionForSpace(spaceName);
 -        setFilters(filters);
++        final Connection conn = connectionForSpace(spaceName);
  
 -        try {
 -            final Connection conn = connectionForThread(schema(spaceName));
 +        initLocalQueryContext(conn, enforceJoinOrder, filters);
  
 +        try {
-             ResultSet rs = executeSqlQueryWithTimer(spaceName, conn, qry, params, true);
+             final PreparedStatement stmt = preparedStatementWithParams(conn, qry, params, true);
  
-             List<GridQueryFieldMetadata> meta = null;
+             List<GridQueryFieldMetadata> meta;
  
-             if (rs != null) {
-                 try {
-                     meta = meta(rs.getMetaData());
-                 }
-                 catch (SQLException e) {
-                     throw new IgniteCheckedException("Failed to get meta data.", e);
-                 }
+             try {
+                 meta = meta(stmt.getMetaData());
+             }
+             catch (SQLException e) {
+                 throw new IgniteCheckedException("Cannot prepare query metadata", e);
              }
  
-             return new GridQueryFieldsResultAdapter(meta, new FieldsIterator(rs));
+             return new GridQueryFieldsResultAdapter(meta, null) {
+                 @Override public GridCloseableIterator<List<?>> iterator() throws IgniteCheckedException{
+                     ResultSet rs = executeSqlQueryWithTimer(spaceName, stmt, conn, qry, params, timeout, cancel);
+ 
+                     return new FieldsIterator(rs);
+                 }
+             };
          }
          finally {
 -            setFilters(null);
 +            GridH2QueryContext.clearThreadLocal();
          }
      }
  
@@@ -877,12 -851,54 +886,52 @@@
  
          bindParameters(stmt, params);
  
+         return stmt;
+     }
+ 
+     /**
+      * Executes sql query statement.
+      *
+      * @param conn Connection,.
+      * @param stmt Statement.
+      * @param cancel Query cancel.
+      * @return Result.
+      * @throws IgniteCheckedException If failed.
+      */
+     private ResultSet executeSqlQuery(final Connection conn, final PreparedStatement stmt,
+         int timeoutMillis, @Nullable GridQueryCancel cancel)
+         throws IgniteCheckedException {
+ 
+         if (timeoutMillis > 0)
+             ((Session)((JdbcConnection)conn).getSession()).setQueryTimeout(timeoutMillis);
+ 
+         if (cancel != null) {
+             cancel.set(new Runnable() {
+                 @Override public void run() {
+                     try {
+                         stmt.cancel();
 -                    } catch (SQLException ignored) {
++                    }
++                    catch (SQLException ignored) {
+                         // No-op.
+                     }
+                 }
+             });
+         }
+ 
          try {
              return stmt.executeQuery();
          }
          catch (SQLException e) {
+             // Throw special exception.
+             if (e.getErrorCode() == ErrorCode.STATEMENT_WAS_CANCELED)
+                 throw new QueryCancelledException();
+ 
              throw new IgniteCheckedException("Failed to execute SQL query.", e);
          }
+         finally {
 -            if(cancel != null)
 -                cancel.setCompleted();
 -
+             if (timeoutMillis > 0)
+                 ((Session)((JdbcConnection)conn).getSession()).setQueryTimeout(0);
+         }
      }
  
      /**
@@@ -982,14 -1034,10 +1057,14 @@@
          if (tbl == null)
              throw new CacheException("Failed to find SQL table for type: " + type.name());
  
 -        setFilters(filters);
 +        String sql = generateQuery(qry, tbl);
 +
 +        Connection conn = connectionForThread(tbl.schemaName());
 +
 +        initLocalQueryContext(conn, false, filter);
  
          try {
-             ResultSet rs = executeSqlQueryWithTimer(spaceName, conn, sql, params, true);
 -            ResultSet rs = executeQuery(spaceName, qry, params, tbl, null);
++            ResultSet rs = executeSqlQueryWithTimer(spaceName, conn, sql, params, true, 0, null);
  
              return new KeyValIterator(rs);
          }
@@@ -998,18 -1046,20 +1073,20 @@@
          }
      }
  
 -    /** {@inheritDoc} */
 -    private Iterable<List<?>> doQueryTwoStep(final GridCacheContext<?, ?> cctx, final GridCacheTwoStepQuery qry,
 -        final boolean keepCacheObj,
 +    /**
 +     * @param cctx Cache context.
 +     * @param qry Query.
 +     * @param keepCacheObj Flag to keep cache object.
 +     * @param enforceJoinOrder Enforce join order of tables.
 +     * @return Iterable result.
 +     */
 +    private Iterable<List<?>> runQueryTwoStep(final GridCacheContext<?,?> cctx, final GridCacheTwoStepQuery qry,
-         final boolean keepCacheObj, final boolean enforceJoinOrder) {
++        final boolean keepCacheObj, final boolean enforceJoinOrder,
+         final int timeoutMillis,
+         final GridQueryCancel cancel) {
          return new Iterable<List<?>>() {
              @Override public Iterator<List<?>> iterator() {
-                 return rdcQryExec.query(cctx, qry, keepCacheObj, enforceJoinOrder);
 -                try {
 -                    return rdcQryExec.query(cctx, qry, keepCacheObj, timeoutMillis, cancel);
 -                }
 -                finally {
 -                    if (cancel != null)
 -                        cancel.setCompleted();
 -                }
++                return rdcQryExec.query(cctx, qry, keepCacheObj, enforceJoinOrder, timeoutMillis, cancel);
              }
          };
      }
@@@ -1038,8 -1088,10 +1115,11 @@@
  
          fqry.setArgs(qry.getArgs());
          fqry.setPageSize(qry.getPageSize());
 +        fqry.setDistributedJoins(qry.isDistributedJoins());
  
+         if(qry.getTimeout() > 0)
+             fqry.setTimeout(qry.getTimeout(), TimeUnit.MILLISECONDS);
+ 
          final QueryCursor<List<?>> res = queryTwoStep(cctx, fqry);
  
          final Iterable<Cache.Entry<K, V>> converted = new Iterable<Cache.Entry<K, V>>() {
@@@ -1203,8 -1198,10 +1283,10 @@@
  
          twoStepQry.pageSize(qry.getPageSize());
  
+         GridQueryCancel cancel = new GridQueryCancel();
+ 
          QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(
-             runQueryTwoStep(cctx, twoStepQry, cctx.keepBinary(), enforceJoinOrder));
 -            doQueryTwoStep(cctx, twoStepQry, cctx.keepBinary(), qry.getTimeout(), cancel), cancel);
++            runQueryTwoStep(cctx, twoStepQry, cctx.keepBinary(), enforceJoinOrder, qry.getTimeout(), cancel), cancel);
  
          cursor.fieldsMeta(meta);
  
@@@ -1263,7 -1271,7 +1345,7 @@@
          if (!upper.startsWith("FROM"))
              from = " FROM " + t +
                  (upper.startsWith("WHERE") || upper.startsWith("ORDER") || upper.startsWith("LIMIT") ?
--                " " : " WHERE ");
++                    " " : " WHERE ");
  
          qry = "SELECT " + t + "." + KEY_FIELD_NAME + ", " + t + "." + VAL_FIELD_NAME + from + qry;
  
@@@ -1561,29 -1552,10 +1643,29 @@@
          if (tbl == null)
              return -1;
  
 -        IgniteSpiCloseableIterator<List<?>> iter = execute(spaceName,
 -                "SELECT COUNT(*) FROM " + tbl.fullTableName(), null, null, 0, null).iterator();
 +        Connection conn = connectionForSpace(spaceName);
 +
 +        setupConnection(conn, false, false);
 +
-         ResultSet rs = executeSqlQuery(conn,
-             "SELECT COUNT(*) FROM " + tbl.fullTableName(), null, false);
- 
 +        try {
++            ResultSet rs = executeSqlQuery(conn, prepareStatement(conn, "SELECT COUNT(*) FROM " + tbl.fullTableName(), false),
++                0, null);
++
 +            if (!rs.next())
 +                throw new IllegalStateException();
  
 -        return ((Number)iter.next().get(0)).longValue();
 +            return rs.getLong(1);
 +        }
 +        catch (SQLException e) {
 +            throw new IgniteCheckedException(e);
 +        }
 +    }
 +
 +    /**
 +     * @return Busy lock.
 +     */
 +    public GridSpinBusyLock busyLock() {
 +        return busyLock;
      }
  
      /**
@@@ -1875,11 -1765,11 +1957,11 @@@
                  U.error(log, "Failed to drop schema on cache stop (will ignore): " + U.maskName(ccfg.getName()), e);
              }
  
 -            for (Iterator<Map.Entry<T3<String, String, Boolean>, TwoStepCachedQuery>> it = twoStepCache.entrySet().iterator();
 -                 it.hasNext();) {
 -                Map.Entry<T3<String, String, Boolean>, TwoStepCachedQuery> e = it.next();
 +            for (Iterator<Map.Entry<TwoStepCachedQueryKey, TwoStepCachedQuery>> it = twoStepCache.entrySet().iterator();
-                  it.hasNext();) {
++                it.hasNext();) {
 +                Map.Entry<TwoStepCachedQueryKey, TwoStepCachedQuery> e = it.next();
  
 -                if (F.eq(e.getKey().get1(), ccfg.getName()))
 +                if (F.eq(e.getKey().space, ccfg.getName()))
                      it.remove();
              }
          }
@@@ -3000,4 -2715,10 +3082,10 @@@
              lastUsage = U.currentTimeMillis();
          }
      }
- }
+ 
+     /** {@inheritDoc} */
+     @Override public void cancelAllQueries() {
+         for (Connection conn : conns)
+             U.close(conn, log);
+     }
 -}
++}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5fac786b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index bb5e419,1f05bf7..0314b3d
--- 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
@@@ -35,6 -32,6 +35,7 @@@ import javax.cache.CacheException
  import org.apache.ignite.IgniteCheckedException;
  import org.apache.ignite.IgniteException;
  import org.apache.ignite.IgniteLogger;
++import org.apache.ignite.cache.query.QueryCancelledException;
  import org.apache.ignite.cluster.ClusterNode;
  import org.apache.ignite.events.CacheQueryExecutedEvent;
  import org.apache.ignite.events.CacheQueryReadEvent;
@@@ -51,12 -49,9 +52,13 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionsReservation;
  import org.apache.ignite.internal.processors.cache.distributed.dht.GridReservable;
  import org.apache.ignite.internal.processors.cache.query.CacheQueryType;
 +import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable;
  import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery;
+ import org.apache.ignite.internal.processors.query.GridQueryCancel;
  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;
  import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest;
@@@ -158,8 -144,8 +160,8 @@@ public class GridMapQueryExecutor 
                  if (nodeRess == null)
                      return;
  
 -                for (QueryResults ress : nodeRess.values())
 -                    ress.cancel();
 +                for (QueryResults ress : nodeRess.results().values())
-                     ress.cancel();
++                    ress.cancel(true);
              }
          }, EventType.EVT_NODE_FAILED, EventType.EVT_NODE_LEFT);
  
@@@ -237,7 -208,7 +239,7 @@@
          if (results == null)
              return;
  
--        results.cancel();
++        results.cancel(true);
      }
  
      /**
@@@ -409,226 -393,132 +411,231 @@@
       * @param req Query request.
       */
      private void onQueryRequest(ClusterNode node, GridQueryRequest req) {
 -        ConcurrentMap<Long,QueryResults> nodeRess = resultsForNode(node.id());
 +        List<Integer> cacheIds;
  
 -        QueryResults qr = null;
 +        if (req.extraSpaces() != null) {
 +            cacheIds = new ArrayList<>(req.extraSpaces().size() + 1);
  
 -        List<GridReservable> reserved = new ArrayList<>();
 +            cacheIds.add(CU.cacheId(req.space()));
  
 -        try {
 -            // Unmarshall query params.
 -            Collection<GridCacheSqlQuery> qrys;
 +            for (String extraSpace : req.extraSpaces())
 +                cacheIds.add(CU.cacheId(extraSpace));
 +        }
 +        else
 +            cacheIds = Collections.singletonList(CU.cacheId(req.space()));
 +
 +        onQueryRequest0(node,
 +            req.requestId(),
 +            req.queries(),
 +            cacheIds,
 +            req.topologyVersion(),
 +            null,
 +            req.partitions(),
 +            null,
 +            req.pageSize(),
-             false);
++            false,
++            req.timeout());
 +    }
  
 -            try {
 -                qrys = req.queries();
 +    /**
 +     * @param node Node.
 +     * @param req Query request.
 +     */
 +    private void onQueryRequest(ClusterNode node, GridH2QueryRequest req) {
 +        Map<UUID,int[]> partsMap = req.partitions();
 +        int[] parts = partsMap == null ? null : partsMap.get(ctx.localNodeId());
 +
 +        onQueryRequest0(node,
 +            req.requestId(),
 +            req.queries(),
 +            req.caches(),
 +            req.topologyVersion(),
 +            partsMap,
 +            parts,
 +            req.tables(),
 +            req.pageSize(),
-             req.isFlagSet(GridH2QueryRequest.FLAG_DISTRIBUTED_JOINS));
++            req.isFlagSet(GridH2QueryRequest.FLAG_DISTRIBUTED_JOINS),
++            req.timeout());
 +    }
  
 -                if (!node.isLocal()) {
 -                    Marshaller m = ctx.config().getMarshaller();
 +    /**
 +     * @param node Node authored request.
 +     * @param reqId Request ID.
 +     * @param qrys Queries to execute.
 +     * @param cacheIds Caches which will be affected by these queries.
 +     * @param topVer Topology version.
 +     * @param partsMap Partitions map for unstable topology.
 +     * @param parts Explicit partitions for current node.
 +     * @param tbls Tables.
 +     * @param pageSize Page size.
 +     * @param distributedJoins Can we expect distributed joins to be ran.
 +     */
 +    private void onQueryRequest0(
 +        ClusterNode node,
 +        long reqId,
 +        Collection<GridCacheSqlQuery> qrys,
 +        List<Integer> cacheIds,
 +        AffinityTopologyVersion topVer,
 +        Map<UUID, int[]> partsMap,
 +        int[] parts,
 +        Collection<String> tbls,
 +        int pageSize,
-         boolean distributedJoins
++        boolean distributedJoins,
++        int timeout
 +    ) {
 +        // Prepare to run queries.
 +        GridCacheContext<?, ?> mainCctx = ctx.cache().context().cacheContext(cacheIds.get(0));
  
 -                    for (GridCacheSqlQuery qry : qrys)
 -                        qry.unmarshallParams(m, ctx);
 -                }
 -            }
 -            catch (IgniteCheckedException e) {
 -                throw new CacheException("Failed to unmarshall parameters.", e);
 -            }
 +        if (mainCctx == null)
 +            throw new CacheException("Failed to find cache.");
 +
 +        NodeResults nodeRess = resultsForNode(node.id());
  
 -            List<String> caches = (List<String>)F.concat(true, req.space(), req.extraSpaces());
 +        QueryResults qr = null;
  
 -            // Topology version can be null in rolling restart with previous version!
 -            final AffinityTopologyVersion topVer = req.topologyVersion();
 +        List<GridReservable> reserved = new ArrayList<>();
  
 +        try {
              if (topVer != null) {
                  // Reserve primary for topology version or explicit partitions.
 -                if (!reservePartitions(caches, topVer, req.partitions(), reserved)) {
 -                    sendRetry(node, req.requestId());
 +                if (!reservePartitions(cacheIds, topVer, parts, reserved)) {
 +                    sendRetry(node, reqId);
  
                      return;
                  }
              }
  
 -            // Prepare to run queries.
 -            GridCacheContext<?,?> mainCctx = cacheContext(req.space());
 +            qr = new QueryResults(reqId, qrys.size(), mainCctx);
  
 -            if (mainCctx == null)
 -                throw new CacheException("Failed to find cache: " + req.space());
 +            if (nodeRess.results().put(reqId, qr) != null)
 +                throw new IllegalStateException();
  
 -            qr = new QueryResults(req.requestId(), qrys.size(), mainCctx);
 +            // Prepare query context.
 +            GridH2QueryContext qctx = new GridH2QueryContext(ctx.localNodeId(),
 +                node.id(),
 +                reqId,
 +                mainCctx.isReplicated() ? REPLICATED : MAP)
 +                .filter(h2.backupFilter(topVer, parts))
 +                .partitionsMap(partsMap)
 +                .distributedJoins(distributedJoins)
 +                .pageSize(pageSize)
 +                .topologyVersion(topVer)
 +                .reservations(reserved);
  
 -            if (nodeRess.put(req.requestId(), qr) != null)
 -                throw new IllegalStateException();
 +            List<GridH2Table> snapshotedTbls = null;
  
 -            h2.setFilters(h2.backupFilter(caches, topVer, req.partitions()));
 +            if (!F.isEmpty(tbls)) {
 +                snapshotedTbls = new ArrayList<>(tbls.size());
  
 -            // TODO Prepare snapshots for all the needed tables before the run.
 +                for (String identifier : tbls) {
 +                    GridH2Table tbl = h2.dataTable(identifier);
  
 -            // Run queries.
 -            int i = 0;
 +                    Objects.requireNonNull(tbl, identifier);
  
 -            for (GridCacheSqlQuery qry : qrys) {
 -                ResultSet rs = h2.executeSqlQueryWithTimer(req.space(),
 -                        h2.connectionForSpace(req.space()),
 -                        qry.query(),
 -                        F.asList(qry.parameters()),
 -                        true,
 -                        req.timeout(),
 -                        qr.cancels[i]);
 +                    tbl.snapshotIndexes(qctx);
  
 -                if (ctx.event().isRecordable(EVT_CACHE_QUERY_EXECUTED)) {
 -                    ctx.event().record(new CacheQueryExecutedEvent<>(
 -                        node,
 -                        "SQL query executed.",
 -                        EVT_CACHE_QUERY_EXECUTED,
 -                        CacheQueryType.SQL.name(),
 -                        mainCctx.namex(),
 -                        null,
 -                        qry.query(),
 -                        null,
 -                        null,
 -                        qry.parameters(),
 -                        node.id(),
 -                        null));
 +                    snapshotedTbls.add(tbl);
                  }
 +            }
  
 -                assert rs instanceof JdbcResultSet : rs.getClass();
 +            Connection conn = h2.connectionForSpace(mainCctx.name());
  
 -                qr.addResult(i, qry, node.id(), rs);
 +            // Here we enforce join order to have the same behavior on all the nodes.
 +            h2.setupConnection(conn, distributedJoins, true);
  
 -                if (qr.canceled) {
 -                    qr.result(i).close();
 +            GridH2QueryContext.set(qctx);
  
 -                    return;
 +            // qctx is set, we have to release reservations inside of it.
 +            reserved = null;
 +
 +            try {
 +                if (nodeRess.cancelled(reqId)) {
 +                    GridH2QueryContext.clear(ctx.localNodeId(), node.id(), reqId, qctx.type());
 +
 +                    nodeRess.results().remove(reqId);
 +
-                     return;
++                    throw new QueryCancelledException();
                  }
  
 -                // Send the first page.
 -                sendNextPage(nodeRess, node, qr, i, req.pageSize());
 +                // Run queries.
 +                int i = 0;
 +
 +                boolean evt = ctx.event().isRecordable(EVT_CACHE_QUERY_EXECUTED);
 +
 +                for (GridCacheSqlQuery qry : qrys) {
 +                    ResultSet rs = h2.executeSqlQueryWithTimer(mainCctx.name(), conn, qry.query(),
-                         F.asList(qry.parameters()), true);
++                        F.asList(qry.parameters()), true,
++                        timeout,
++                        qr.cancels[i]);
 +
 +                    if (evt) {
 +                        ctx.event().record(new CacheQueryExecutedEvent<>(
 +                            node,
 +                            "SQL query executed.",
 +                            EVT_CACHE_QUERY_EXECUTED,
 +                            CacheQueryType.SQL.name(),
 +                            mainCctx.namex(),
 +                            null,
 +                            qry.query(),
 +                            null,
 +                            null,
 +                            qry.parameters(),
 +                            node.id(),
 +                            null));
 +                    }
 +
 +                    assert rs instanceof JdbcResultSet : rs.getClass();
 +
 +                    qr.addResult(i, qry, node.id(), rs);
 +
 +                    if (qr.canceled) {
 +                        qr.result(i).close();
  
-                         return;
 -                i++;
++                        throw new QueryCancelledException();
 +                    }
 +
 +                    // Send the first page.
 +                    sendNextPage(nodeRess, node, qr, i, pageSize);
 +
 +                    i++;
 +                }
 +            }
 +            finally {
 +                GridH2QueryContext.clearThreadLocal();
 +
 +                if (!distributedJoins)
 +                    qctx.clearContext(false);
 +
 +                if (!F.isEmpty(snapshotedTbls)) {
 +                    for (GridH2Table dataTbl : snapshotedTbls)
 +                        dataTbl.releaseSnapshots();
 +                }
              }
          }
          catch (Throwable e) {
              if (qr != null) {
 -                nodeRess.remove(req.requestId(), qr);
 +                nodeRess.results().remove(reqId, qr);
  
--                qr.cancel();
++                qr.cancel(false);
              }
  
 -            U.error(log, "Failed to execute local query: " + req, e);
 +            if (X.hasCause(e, GridH2RetryException.class))
 +                sendRetry(node, reqId);
 +            else {
 +                U.error(log, "Failed to execute local query.", e);
  
 -            sendError(node, req.requestId(), e);
 +                sendError(node, reqId, e);
  
 -            if (e instanceof Error)
 -                throw (Error)e;
 +                if (e instanceof Error)
 +                    throw (Error)e;
 +            }
          }
          finally {
 -            h2.setFilters(null);
 -
 -            // Release reserved partitions.
 -            for (GridReservable r : reserved)
 -                r.release();
 -
 -            // Ensure all cancels state is correct.
 -            if (qr != null)
 -                for (int i = 0; i < qr.cancels.length; i++) {
 -                    GridQueryCancel cancel = qr.cancels[i];
 -
 -                    if (cancel != null)
 -                        cancel.setCompleted();
 -                }
 +            if (reserved != null) {
 +                // Release reserved partitions.
 +                for (int i = 0; i < reserved.size(); i++)
 +                    reserved.get(i).release();
 +            }
          }
      }
  
@@@ -661,12 -548,12 +668,24 @@@
       * @param req Request.
       */
      private void onNextPageRequest(ClusterNode node, GridQueryNextPageRequest req) {
 -        ConcurrentMap<Long, QueryResults> nodeRess = qryRess.get(node.id());
 +        NodeResults nodeRess = qryRess.get(node.id());
 +
-         QueryResults qr = nodeRess == null ? null : nodeRess.results().get(req.queryRequestId());
++        if (nodeRess == null) {
++            sendError(node, req.queryRequestId(), new CacheException("No node result found for request: " + req));
  
-         if (qr == null || qr.canceled)
 -        QueryResults qr = nodeRess == null ? null : nodeRess.get(req.queryRequestId());
++            return;
++        } else if (nodeRess.cancelled(req.queryRequestId())) {
++            sendError(node, req.queryRequestId(), new QueryCancelledException());
++
++            return;
++        }
+ 
 -        if (qr == null || qr.canceled)
++        QueryResults qr = nodeRess.results().get(req.queryRequestId());
++
++        if (qr == null)
              sendError(node, req.queryRequestId(), new CacheException("No query result found for request: " + req));
++        else if (qr.canceled)
++            sendError(node, req.queryRequestId(), new QueryCancelledException());
          else
              sendNextPage(nodeRess, node, qr, req.query(), req.pageSize());
      }
@@@ -854,9 -705,9 +880,9 @@@
          }
  
          /**
-          *
+          * Cancels the query.
           */
--        void cancel() {
++        void cancel(boolean forceQryCancel) {
              if (canceled)
                  return;
  
@@@ -865,8 -716,16 +891,18 @@@
              for (int i = 0; i < results.length(); i++) {
                  QueryResult res = results.get(i);
  
-                 if (res != null)
+                 if (res != null) {
                      res.close();
+ 
+                     continue;
+                 }
+ 
 -                GridQueryCancel cancel = cancels[i];
++                if (forceQryCancel) {
++                    GridQueryCancel cancel = cancels[i];
+ 
 -                if (cancel != null)
 -                    cancel.cancel();
++                    if (cancel != null)
++                        cancel.cancel();
++                }
              }
          }
      }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5fac786b/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 04449ac,3fdbf42..3847373
--- 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
@@@ -46,6 -45,6 +45,7 @@@ import org.apache.ignite.IgniteCheckedE
  import org.apache.ignite.IgniteClientDisconnectedException;
  import org.apache.ignite.IgniteException;
  import org.apache.ignite.IgniteLogger;
++import org.apache.ignite.cache.query.QueryCancelledException;
  import org.apache.ignite.cluster.ClusterNode;
  import org.apache.ignite.events.DiscoveryEvent;
  import org.apache.ignite.events.Event;
@@@ -64,9 -62,10 +64,10 @@@ import org.apache.ignite.internal.proce
  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.GridQueryCacheObjectsIterator;
+ import org.apache.ignite.internal.processors.query.GridQueryCancel;
  import org.apache.ignite.internal.processors.query.h2.GridH2ResultSetIterator;
  import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 -import org.apache.ignite.cache.query.QueryCancelledException;
 +import org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryContext;
  import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuerySplitter;
  import org.apache.ignite.internal.processors.query.h2.sql.GridSqlType;
  import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest;
@@@ -74,14 -73,11 +75,15 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest;
  import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageResponse;
  import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryRequest;
 +import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryRequest;
  import org.apache.ignite.internal.util.GridSpinBusyLock;
 +import org.apache.ignite.internal.util.typedef.CIX2;
  import org.apache.ignite.internal.util.typedef.F;
++import org.apache.ignite.internal.util.typedef.X;
  import org.apache.ignite.internal.util.typedef.internal.U;
 +import org.apache.ignite.lang.IgniteBiClosure;
  import org.apache.ignite.lang.IgniteFuture;
 -import org.apache.ignite.marshaller.Marshaller;
 +import org.apache.ignite.lang.IgniteProductVersion;
  import org.apache.ignite.plugin.extensions.communication.Message;
  import org.h2.command.ddl.CreateTableData;
  import org.h2.engine.Session;
@@@ -468,16 -456,13 +476,20 @@@ public class GridReduceQueryExecutor 
      /**
       * @param cctx Cache context.
       * @param qry Query.
 -     * @param keepBinary Keep binary.
 +     * @param keepPortable Keep portable.
 +     * @param enforceJoinOrder Enforce join order of tables.
-      * @return Cursor.
+      * @param timeoutMillis Timeout in milliseconds.
+      * @param cancel Query cancel.
+      * @return Rows iterator.
       */
 -    public Iterator<List<?>> query(GridCacheContext<?,?> cctx, GridCacheTwoStepQuery qry, boolean keepBinary,
 -        int timeoutMillis, GridQueryCancel cancel) {
 +    public Iterator<List<?>> query(
 +        GridCacheContext<?, ?> cctx,
 +        GridCacheTwoStepQuery qry,
 +        boolean keepPortable,
-         boolean enforceJoinOrder
++        boolean enforceJoinOrder,
++        int timeoutMillis,
++        GridQueryCancel cancel
 +    ) {
          for (int attempt = 0;; attempt++) {
              if (attempt != 0) {
                  try {
@@@ -579,39 -575,27 +595,49 @@@
                          mapQrys.add(new GridCacheSqlQuery("EXPLAIN " + mapQry.query(), mapQry.parameters()));
                  }
  
-                 boolean retry = false;
- 
 -                if (nodes.size() != 1 || !F.first(nodes).isLocal()) { // Marshall params for remotes.
 -                    Marshaller m = ctx.config().getMarshaller();
 +                IgniteProductVersion minNodeVer = cctx.shared().exchange().minimumNodeVersion(topVer);
  
 -                    for (GridCacheSqlQuery mapQry : mapQrys)
 -                        mapQry.marshallParams(m);
 -                }
 +                final boolean oldStyle = minNodeVer.compareToIgnoreTimestamp(DISTRIBUTED_JOIN_SINCE) < 0;
 +                final boolean distributedJoins = qry.distributedJoins();
  
+                 cancel.set(new Runnable() {
+                     @Override public void run() {
 -                        send(finalNodes, new GridQueryCancelRequest(qryReqId), null);
++                        send(finalNodes, new GridQueryCancelRequest(qryReqId), null, false);
+                     }
+                 });
+ 
+                 boolean retry = false;
+ 
 +                if (oldStyle && distributedJoins)
 +                    throw new CacheException("Failed to enable distributed joins. Topology contains older data nodes.");
 +
                  if (send(nodes,
 -                    new GridQueryRequest(qryReqId, r.pageSize, space, mapQrys, topVer, extraSpaces, null, timeoutMillis), partsMap)) {
 +                    oldStyle ?
 +                        new GridQueryRequest(qryReqId,
 +                            r.pageSize,
 +                            space,
 +                            mapQrys,
 +                            topVer,
 +                            extraSpaces(space, qry.spaces()),
-                             null) :
++                            null,
++                            timeoutMillis) :
 +                        new GridH2QueryRequest()
 +                            .requestId(qryReqId)
 +                            .topologyVersion(topVer)
 +                            .pageSize(r.pageSize)
 +                            .caches(qry.caches())
 +                            .tables(distributedJoins ? qry.tables() : null)
 +                            .partitions(convert(partsMap))
 +                            .queries(mapQrys)
-                             .flags(distributedJoins ? GridH2QueryRequest.FLAG_DISTRIBUTED_JOINS : 0),
++                            .flags(distributedJoins ? GridH2QueryRequest.FLAG_DISTRIBUTED_JOINS : 0)
++                            .timeout(timeoutMillis),
 +                    oldStyle && partsMap != null ? new ExplicitPartitionsSpecializer(partsMap) : null,
 +                    distributedJoins)
-                 ) {
++                    ) {
                      awaitAllReplies(r, nodes);
  
+                     cancel.checkCancelled();
+ 
                      Object state = r.state.get();
  
                      if (state != null) {
@@@ -663,30 -653,20 +692,34 @@@
                          resIter = res.iterator();
                      }
                      else {
+                         cancel.checkCancelled();
+ 
 -                        GridCacheSqlQuery rdc = qry.reduceQuery();
 +                        UUID locNodeId = ctx.localNodeId();
 +
 +                        h2.setupConnection(r.conn, false, enforceJoinOrder);
  
 -                        // Statement caching is prohibited here because we can't guarantee correct merge index reuse.
 -                        ResultSet res = h2.executeSqlQueryWithTimer(space,
 -                            r.conn,
 -                            rdc.query(),
 -                            F.asList(rdc.parameters()),
 -                            false,
 -                            timeoutMillis,
 -                            cancel);
 +                        GridH2QueryContext.set(new GridH2QueryContext(locNodeId, locNodeId, qryReqId, REDUCE)
 +                            .pageSize(r.pageSize).distributedJoins(false));
  
 -                        resIter = new Iter(res);
 +                        try {
 +                            if (qry.explain())
 +                                return explainPlan(r.conn, space, qry);
 +
 +                            GridCacheSqlQuery rdc = qry.reduceQuery();
 +
 +                            ResultSet res = h2.executeSqlQueryWithTimer(space,
 +                                r.conn,
 +                                rdc.query(),
 +                                F.asList(rdc.parameters()),
-                                 false);
++                                false,
++                                timeoutMillis,
++                                cancel);
 +
 +                            resIter = new Iter(res);
 +                        }
 +                        finally {
 +                            GridH2QueryContext.clearThreadLocal();
 +                        }
                      }
                  }
  
@@@ -699,16 -677,7 +730,7 @@@
                      continue;
                  }
  
-                 final Collection<ClusterNode> finalNodes = nodes;
- 
-                 return new GridQueryCacheObjectsIterator(resIter, cctx, keepPortable) {
-                     @Override public void close() throws Exception {
-                         super.close();
- 
-                         if (distributedJoins || !allIndexesFetched(r.idxs))
-                             send(finalNodes, new GridQueryCancelRequest(qryReqId), null, false);
-                     }
-                 };
 -                return new GridQueryCacheObjectsIterator(resIter, cctx, keepBinary);
++                return new GridQueryCacheObjectsIterator(resIter, cctx, keepPortable);
              }
              catch (IgniteCheckedException | RuntimeException e) {
                  U.closeQuiet(r.conn);
@@@ -741,19 -717,33 +770,43 @@@
      }
  
      /**
 +     * @param idxs Merge indexes.
 +     * @return {@code true} If all remote data was fetched.
 +     */
 +    private static boolean allIndexesFetched(List<GridMergeIndex> idxs) {
 +        for (int i = 0; i <  idxs.size(); i++) {
 +            if (!idxs.get(i).fetchedAll())
 +                return false;
 +        }
 +
 +        return true;
 +    }
 +
 +    /**
+      * Returns true if the exception is triggered by query cancel.
+      *
+      * @param e Exception.
+      * @return {@code true} if exception is caused by cancel.
+      */
+     private boolean wasCancelled(CacheException e) {
 -        return e.getSuppressed() != null && e.getSuppressed().length > 0 &&
 -            e.getSuppressed()[0] instanceof QueryCancelledException;
++        return X.hasSuppressed(e, QueryCancelledException.class);
+     }
+ 
+     /**
 -     * Explicitly cancels remote queries.
 -     * @param nodes Nodes.
+      * @param r Query run.
+      * @param qryReqId Query id.
+      */
+     private void cancelRemoteQueriesIfNeeded(Collection<ClusterNode> nodes, QueryRun r, long qryReqId) {
+         for (GridMergeIndex idx : r.idxs) {
+             if (!idx.fetchedAll()) {
 -                send(nodes, new GridQueryCancelRequest(qryReqId), null);
++                send(nodes, new GridQueryCancelRequest(qryReqId), null, false);
+ 
+                 break;
+             }
+         }
+     }
+ 
+     /**
       * @param r Query run.
       * @param nodes Nodes to check periodically if they alive.
       * @throws IgniteInterruptedCheckedException If interrupted.
@@@ -1290,7 -1259,7 +1345,7 @@@
                  latch.countDown();
  
              for (GridMergeIndex idx : idxs) // Fail all merge indexes.
--                idx.fail(nodeId);
++                idx.fail(nodeId, o instanceof CacheException ? (CacheException) o : null);
          }
  
          /**
@@@ -1332,24 -1301,4 +1387,24 @@@
              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/5fac786b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java
index 0000000,0000000..be34a09
new file mode 100644
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java
@@@ -1,0 -1,0 +1,7 @@@
++package org.apache.ignite.internal.processors.cache.distributed.near;
++
++/**
++ * Created by vozerov on 31.10.2016.
++ */
++public class IgniteCacheQueryAbstractDistributedJoinSelfTest {
++}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5fac786b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java
index 0000000,0000000..80bd62e
new file mode 100644
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java
@@@ -1,0 -1,0 +1,7 @@@
++package org.apache.ignite.internal.processors.cache.distributed.near;
++
++/**
++ * Created by vozerov on 31.10.2016.
++ */
++public class IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest {
++}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5fac786b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index 760ee19,6e493ea..bcf8f9d
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@@ -349,8 -348,8 +349,8 @@@ public abstract class GridIndexingSpiAb
  
          // Fields query
          GridQueryFieldsResult fieldsRes =
 -            spi.execute("A", "select a.a.name n1, a.a.age a1, b.a.name n2, " +
 -                "b.a.age a2 from a.a, b.a where a.a.id = b.a.id ", Collections.emptySet(), null, 0, null);
 +            spi.queryLocalSqlFields("A", "select a.a.name n1, a.a.age a1, b.a.name n2, " +
-             "b.a.age a2 from a.a, b.a where a.a.id = b.a.id ", Collections.emptySet(), null, false);
++            "b.a.age a2 from a.a, b.a where a.a.id = b.a.id ", Collections.emptySet(), null, false, 0, null);
  
          String[] aliases = {"N1", "A1", "N2", "A2"};
          Object[] vals = { "Valera", 19, "Kolya", 25};
@@@ -451,8 -450,7 +451,8 @@@
                  time = now;
                  range *= 3;
  
 -                GridQueryFieldsResult res = spi.execute("A", sql, Arrays.<Object>asList(1, range), null, 0, null);
 +                GridQueryFieldsResult res = spi.queryLocalSqlFields("A", sql, Arrays.<Object>asList(1, range), null,
-                     false);
++                    false, 0, null);
  
                  assert res.iterator().hasNext();
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/5fac786b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
index 568f880,9128f76..5722c01
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
@@@ -39,7 -41,12 +41,8 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest;
  import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQuerySelfTest;
  import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest;
+ import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQueryCancelOrTimeoutSelfTest;
  import org.apache.ignite.internal.processors.cache.query.GridCacheSwapScanQuerySelfTest;
 -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.h2.sql.BaseH2CompareQueryTest;
  import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryTest;
  import org.apache.ignite.spi.communication.tcp.GridOrderedMessageCancelSelfTest;


[39/50] [abbrv] ignite git commit: .NET: Fix code analysis warnings in EntityFramework component

Posted by vo...@apache.org.
.NET: Fix code analysis warnings in EntityFramework component


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

Branch: refs/heads/ignite-2693
Commit: 767a8a3375f8db50c660e076e885f92e27b324a3
Parents: 058ad50
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Thu Nov 10 10:34:59 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu Nov 10 10:34:59 2016 +0300

----------------------------------------------------------------------
 .../EntityFrameworkCacheTest.cs                 | 20 ++++++++++++--------
 1 file changed, 12 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/767a8a33/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheTest.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheTest.cs
index cfc9f66..5fbd8fe 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheTest.cs
@@ -665,7 +665,7 @@ namespace Apache.Ignite.EntityFramework.Tests
             TestUtils.RunMultiThreaded(CreateRemoveBlog, 4, 20);
 
             // Wait for the cleanup to complete.
-            Thread.Sleep(200);
+            Thread.Sleep(500);
 
             // Only one version of data is in the cache.
             Assert.AreEqual(1, _cache.GetSize());
@@ -908,7 +908,7 @@ namespace Apache.Ignite.EntityFramework.Tests
             }
         }
 
-        private class DelegateCachingPolicy : IDbCachingPolicy
+        private class DelegateCachingPolicy : DbCachingPolicy
         {
             public Func<DbQueryInfo, bool> CanBeCachedFunc { get; set; }
 
@@ -918,24 +918,28 @@ namespace Apache.Ignite.EntityFramework.Tests
 
             public Func<DbQueryInfo, DbCachingMode> GetCachingStrategyFunc { get; set; }
 
-            public bool CanBeCached(DbQueryInfo queryInfo)
+            public override bool CanBeCached(DbQueryInfo queryInfo)
             {
                 return CanBeCachedFunc == null || CanBeCachedFunc(queryInfo);
             }
 
-            public bool CanBeCached(DbQueryInfo queryInfo, int rowCount)
+            public override bool CanBeCached(DbQueryInfo queryInfo, int rowCount)
             {
                 return CanBeCachedRowsFunc == null || CanBeCachedRowsFunc(queryInfo, rowCount);
             }
 
-            public TimeSpan GetExpirationTimeout(DbQueryInfo queryInfo)
+            public override TimeSpan GetExpirationTimeout(DbQueryInfo queryInfo)
             {
-                return GetExpirationTimeoutFunc == null ? TimeSpan.MaxValue : GetExpirationTimeoutFunc(queryInfo);
+                return GetExpirationTimeoutFunc == null 
+                    ? base.GetExpirationTimeout(queryInfo) 
+                    : GetExpirationTimeoutFunc(queryInfo);
             }
 
-            public DbCachingMode GetCachingMode(DbQueryInfo queryInfo)
+            public override DbCachingMode GetCachingMode(DbQueryInfo queryInfo)
             {
-                return GetCachingStrategyFunc == null ? DbCachingMode.ReadWrite : GetCachingStrategyFunc(queryInfo);
+                return GetCachingStrategyFunc == null 
+                    ? base.GetCachingMode(queryInfo)
+                    : GetCachingStrategyFunc(queryInfo);
             }
         }
     }


[20/50] [abbrv] ignite git commit: IGNITE-4118 .NET: Optimistic transaction example

Posted by vo...@apache.org.
IGNITE-4118 .NET: Optimistic transaction example

This closes #1200


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

Branch: refs/heads/ignite-2693
Commit: df670c7d64046d282c053f296c47a4743c58c8b1
Parents: ead1519
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Nov 7 12:40:00 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Nov 7 12:40:00 2016 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Examples.csproj               |   1 +
 .../Datagrid/OptimisticTransactionExample.cs    | 110 +++++++++++++++++++
 2 files changed, 111 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/df670c7d/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
index 9b1d323..2ee91da 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
@@ -52,6 +52,7 @@
     <Compile Include="Compute\TaskExample.cs" />
     <Compile Include="Datagrid\ContinuousQueryExample.cs" />
     <Compile Include="Datagrid\DataStreamerExample.cs" />
+    <Compile Include="Datagrid\OptimisticTransactionExample.cs" />
     <Compile Include="Datagrid\PutGetExample.cs" />
     <Compile Include="Datagrid\LinqExample.cs" />
     <Compile Include="Datagrid\BinaryModeExample.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/df670c7d/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/OptimisticTransactionExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/OptimisticTransactionExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/OptimisticTransactionExample.cs
new file mode 100644
index 0000000..2f9d164
--- /dev/null
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/OptimisticTransactionExample.cs
@@ -0,0 +1,110 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+using System;
+using System.Threading;
+using System.Threading.Tasks;
+using Apache.Ignite.Core;
+using Apache.Ignite.Core.Cache;
+using Apache.Ignite.Core.Cache.Configuration;
+using Apache.Ignite.Core.Transactions;
+
+namespace Apache.Ignite.Examples.Datagrid
+{
+    /// <summary>
+    /// This example demonstrates optimistic transaction concurrency control.
+    /// <para />
+    /// 1) Build the project Apache.Ignite.ExamplesDll (select it -> right-click -> Build).
+    ///    Apache.Ignite.ExamplesDll.dll must appear in %IGNITE_HOME%/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/bin/${Platform]/${Configuration} folder.
+    /// 2) Set this class as startup object (Apache.Ignite.Examples project -> right-click -> Properties ->
+    ///     Application -> Startup object);
+    /// 3) Start example (F5 or Ctrl+F5).
+    /// <para />
+    /// This example can be run with standalone Apache Ignite.NET node:
+    /// 1) Run %IGNITE_HOME%/platforms/dotnet/bin/Apache.Ignite.exe:
+    /// Apache.Ignite.exe -configFileName=platforms\dotnet\examples\apache.ignite.examples\app.config -assembly=[path_to_Apache.Ignite.ExamplesDll.dll]
+    /// 2) Start example.
+    /// </summary>
+    public class OptimisticTransactionExample
+    {
+        /// <summary>Cache name.</summary>
+        private const string CacheName = "dotnet_optimistic_tx_example";
+
+        [STAThread]
+        public static void Main()
+        {
+            using (var ignite = Ignition.StartFromApplicationConfiguration())
+            {
+                Console.WriteLine();
+                Console.WriteLine(">>> Optimistic transaction example started.");
+
+                // Create Transactional cache.
+                var cacheCfg = new CacheConfiguration(CacheName) { AtomicityMode = CacheAtomicityMode.Transactional };
+
+                var cache = ignite.GetOrCreateCache<int, int>(cacheCfg);
+
+                // Put a value.
+                cache[1] = 0;
+
+                // Increment a value in parallel within a transaction.
+                var task1 = Task.Factory.StartNew(() => IncrementCacheValue(cache, 1));
+                var task2 = Task.Factory.StartNew(() => IncrementCacheValue(cache, 2));
+
+                Task.WaitAll(task1, task2);
+
+                Console.WriteLine();
+                Console.WriteLine(">>> Resulting value in cache: " + cache[1]);
+
+                Console.WriteLine();
+                Console.WriteLine(">>> Example finished, press any key to exit ...");
+                Console.ReadKey();
+            }
+        }
+
+        /// <summary>
+        /// Increments the cache value within a transaction.
+        /// </summary>
+        /// <param name="cache">The cache.</param>
+        /// <param name="threadId">The thread identifier.</param>
+        private static void IncrementCacheValue(ICache<int, int> cache, int threadId)
+        {
+            try
+            {
+                var transactions = cache.Ignite.GetTransactions();
+
+                using (var tx = transactions.TxStart(TransactionConcurrency.Optimistic,
+                    TransactionIsolation.Serializable))
+                {
+                    // Increment cache value.
+                    cache[1]++;
+
+                    // Introduce a delay to ensure lock conflict.
+                    Thread.Sleep(TimeSpan.FromSeconds(2.5));
+
+                    tx.Commit();
+                }
+
+                Console.WriteLine("\n>>> Thread {0} successfully incremented cached value.", threadId);
+            }
+            catch (TransactionOptimisticException ex)
+            {
+                Console.WriteLine("\n>>> Thread {0} failed to increment cached value. " +
+                                  "Caught an expected optimistic exception: {1}", threadId, ex.Message);
+            }
+        }
+    }
+}
\ No newline at end of file


[35/50] [abbrv] ignite git commit: IGNITE-1915 .NET: Ignite as Entity Framework Second-Level Cache

Posted by vo...@apache.org.
IGNITE-1915 .NET: Ignite as Entity Framework Second-Level Cache


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

Branch: refs/heads/ignite-2693
Commit: 5b31d83f38732c8e92807000be858e19387108fd
Parents: 2bc234e
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Wed Nov 9 18:12:35 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Wed Nov 9 18:12:35 2016 +0300

----------------------------------------------------------------------
 .../dotnet/PlatformDotNetConfigurationEx.java   |  16 +-
 ...PlatformDotNetEntityFrameworkCacheEntry.java | 102 ++
 ...formDotNetEntityFrameworkCacheExtension.java | 353 +++++++
 .../PlatformDotNetEntityFrameworkCacheKey.java  | 164 ++++
 ...EntityFrameworkIncreaseVersionProcessor.java |  45 +
 .../Apache.Ignite.Core.Tests.NuGet.csproj       |  11 +
 .../EntityFrameworkCacheTest.cs                 |  62 ++
 .../packages.config                             |   2 +
 .../Cache/CacheAbstractTest.cs                  |  11 +-
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |   2 +
 .../Impl/Binary/BinaryObjectHeader.cs           |   2 +-
 .../Apache.Ignite.EntityFramework.Tests.csproj  |  96 ++
 .../Apache.Ignite.EntityFramework.Tests.snk     | Bin 0 -> 596 bytes
 .../App.config                                  |  71 ++
 .../ArrayDbDataReaderTests.cs                   | 192 ++++
 .../DbCachingPolicyTest.cs                      |  43 +
 .../EntityFrameworkCacheInitializationTest.cs   | 137 +++
 .../EntityFrameworkCacheTest.cs                 | 942 +++++++++++++++++++
 .../Properties/AssemblyInfo.cs                  |  39 +
 .../packages.config                             |  23 +
 .../Apache.Ignite.EntityFramework.csproj        |  93 ++
 .../Apache.Ignite.EntityFramework.nuspec        |  57 ++
 .../Apache.Ignite.EntityFramework.snk           | Bin 0 -> 596 bytes
 .../DbCachingMode.cs                            |  48 +
 .../DbCachingPolicy.cs                          |  71 ++
 .../DbQueryInfo.cs                              |  78 ++
 .../IDbCachingPolicy.cs                         |  58 ++
 .../IgniteDbConfiguration.cs                    | 240 +++++
 .../Impl/ArrayDbDataReader.cs                   | 305 ++++++
 .../Impl/DataReaderField.cs                     |  74 ++
 .../Impl/DataReaderResult.cs                    |  93 ++
 .../Impl/DbCache.cs                             | 295 ++++++
 .../Impl/DbCacheKey.cs                          |  92 ++
 .../Impl/DbCommandDefinitionProxy.cs            |  51 +
 .../Impl/DbCommandInfo.cs                       | 158 ++++
 .../Impl/DbCommandProxy.cs                      | 263 ++++++
 .../Impl/DbProviderServicesProxy.cs             | 169 ++++
 .../Impl/DbTransactionInterceptor.cs            | 134 +++
 .../Properties/AssemblyInfo.cs                  |  41 +
 .../packages.config                             |  20 +
 modules/platforms/dotnet/Apache.Ignite.sln      |  28 +
 41 files changed, 4673 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationEx.java
index 34e7ce2..8448733 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationEx.java
@@ -21,14 +21,15 @@ import org.apache.ignite.internal.logger.platform.PlatformLogger;
 import org.apache.ignite.internal.processors.platform.PlatformConfigurationEx;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheExtension;
 import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway;
+import org.apache.ignite.internal.processors.platform.entityframework.PlatformDotNetEntityFrameworkCacheExtension;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemoryManagerImpl;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 import org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionCacheExtension;
 import org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration;
 import org.jetbrains.annotations.Nullable;
 
+import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 
 /**
  * Extended .Net configuration.
@@ -83,13 +84,18 @@ public class PlatformDotNetConfigurationEx extends PlatformDotNetConfiguration i
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformLogger logger() {
-        return logger;
+    @Nullable @Override public Collection<PlatformCacheExtension> cacheExtensions() {
+        Collection<PlatformCacheExtension> exts = new ArrayList<>(2);
+
+        exts.add(new PlatformDotNetSessionCacheExtension());
+        exts.add(new PlatformDotNetEntityFrameworkCacheExtension());
+
+        return exts;
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public Collection<PlatformCacheExtension> cacheExtensions() {
-        return Collections.<PlatformCacheExtension>singleton(new PlatformDotNetSessionCacheExtension());
+    @Override public PlatformLogger logger() {
+        return logger;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/entityframework/PlatformDotNetEntityFrameworkCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/entityframework/PlatformDotNetEntityFrameworkCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/entityframework/PlatformDotNetEntityFrameworkCacheEntry.java
new file mode 100644
index 0000000..676b411
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/entityframework/PlatformDotNetEntityFrameworkCacheEntry.java
@@ -0,0 +1,102 @@
+/*
+ * 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.platform.entityframework;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
+
+/**
+ * EntityFramework cache entry.
+ */
+public class PlatformDotNetEntityFrameworkCacheEntry implements Binarylizable {
+    /** Dependent entity set names. */
+    private String[] entitySets;
+
+    /** Cached data bytes. */
+    private byte[] data;
+
+    /**
+     * Ctor.
+     */
+    public PlatformDotNetEntityFrameworkCacheEntry() {
+        // No-op.
+    }
+
+    /**
+     * Ctor.
+     *
+     * @param entitySets Entity set names.
+     * @param data Data bytes.
+     */
+    PlatformDotNetEntityFrameworkCacheEntry(String[] entitySets, byte[] data) {
+        this.entitySets = entitySets;
+        this.data = data;
+    }
+
+    /**
+     * @return Dependent entity sets with versions.
+     */
+    public String[] entitySets() {
+        return entitySets;
+    }
+
+    /**
+     * @return Cached data bytes.
+     */
+    public byte[] data() {
+        return data;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        final BinaryRawWriter raw = writer.rawWriter();
+
+        if (entitySets != null) {
+            raw.writeInt(entitySets.length);
+
+            for (String entitySet : entitySets)
+                raw.writeString(entitySet);
+        }
+        else
+            raw.writeInt(-1);
+
+        raw.writeByteArray(data);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader raw = reader.rawReader();
+
+        int cnt = raw.readInt();
+
+        if (cnt >= 0) {
+            entitySets = new String[cnt];
+
+            for (int i = 0; i < cnt; i++)
+                entitySets[i] = raw.readString();
+        }
+        else
+            entitySets = null;
+
+        data = raw.readByteArray();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/entityframework/PlatformDotNetEntityFrameworkCacheExtension.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/entityframework/PlatformDotNetEntityFrameworkCacheExtension.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/entityframework/PlatformDotNetEntityFrameworkCacheExtension.java
new file mode 100644
index 0000000..d4755de
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/entityframework/PlatformDotNetEntityFrameworkCacheExtension.java
@@ -0,0 +1,353 @@
+/*
+ * 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.platform.entityframework;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteCompute;
+import org.apache.ignite.cache.CachePeekMode;
+import org.apache.ignite.cluster.ClusterGroup;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.binary.BinaryRawReaderEx;
+import org.apache.ignite.internal.processors.platform.cache.PlatformCache;
+import org.apache.ignite.internal.processors.platform.cache.PlatformCacheExtension;
+import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.resources.IgniteInstanceResource;
+
+import javax.cache.Cache;
+import javax.cache.processor.EntryProcessorResult;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * EntityFramework cache extension.
+ */
+@SuppressWarnings("unchecked")
+public class PlatformDotNetEntityFrameworkCacheExtension implements PlatformCacheExtension {
+    /** Extension ID. */
+    private static final int EXT_ID = 1;
+
+    /** Operation: increment entity set versions. */
+    private static final int OP_INVALIDATE_SETS = 1;
+
+    /** Operation: put item async. */
+    private static final int OP_PUT_ITEM = 2;
+
+    /** Operation: get item. */
+    private static final int OP_GET_ITEM = 3;
+
+    /** Cache key for cleanup node ID. */
+    private static final CleanupNodeId CLEANUP_NODE_ID = new CleanupNodeId();
+
+    /** Indicates whether local cleanup is in progress, per cache name. */
+    private final Map<String, Boolean> cleanupFlags = new ConcurrentHashMap<>();
+
+    /** {@inheritDoc} */
+    @Override public int id() {
+        return EXT_ID;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public long processInOutStreamLong(PlatformCache target, int type, BinaryRawReaderEx reader,
+        PlatformMemory mem) throws IgniteCheckedException {
+        switch (type) {
+            case OP_INVALIDATE_SETS: {
+                final IgniteCache<String, Long> metaCache = target.rawCache();
+                final String dataCacheName = reader.readString();
+
+                int cnt = reader.readInt();
+
+                assert cnt > 0;
+
+                final Set<String> entitySetNames = new HashSet(cnt);
+
+                for (int i = 0; i < cnt; i++)
+                    entitySetNames.add(reader.readString());
+
+                final Map<String, EntryProcessorResult<Long>> curVers =
+                    metaCache.invokeAll(entitySetNames, new PlatformDotNetEntityFrameworkIncreaseVersionProcessor());
+
+                if (curVers.size() != cnt)
+                    throw new IgniteCheckedException("Failed to update entity set versions [expected=" + cnt +
+                        ", actual=" + curVers.size() + ']');
+
+                Ignite grid = target.platformContext().kernalContext().grid();
+
+                startBackgroundCleanup(grid, (IgniteCache<CleanupNodeId, UUID>)(IgniteCache)metaCache,
+                    dataCacheName, curVers);
+
+                return target.writeResult(mem, null);
+            }
+
+            case OP_PUT_ITEM: {
+                String query = reader.readString();
+
+                long[] versions = null;
+                String[] entitySets = null;
+
+                int cnt = reader.readInt();
+
+                if (cnt >= 0) {
+                    versions = new long[cnt];
+                    entitySets = new String[cnt];
+
+                    for (int i = 0; i < cnt; i++) {
+                        versions[i] = reader.readLong();
+                        entitySets[i] = reader.readString();
+                    }
+                }
+
+                byte[] data = reader.readByteArray();
+
+                PlatformDotNetEntityFrameworkCacheEntry efEntry =
+                    new PlatformDotNetEntityFrameworkCacheEntry(entitySets, data);
+
+                IgniteCache<PlatformDotNetEntityFrameworkCacheKey, PlatformDotNetEntityFrameworkCacheEntry> dataCache
+                    = target.rawCache();
+
+                PlatformDotNetEntityFrameworkCacheKey key = new PlatformDotNetEntityFrameworkCacheKey(query, versions);
+
+                dataCache.put(key, efEntry);
+
+                return target.writeResult(mem, null);
+            }
+
+            case OP_GET_ITEM: {
+                String query = reader.readString();
+
+                long[] versions = null;
+
+                int cnt = reader.readInt();
+
+                if (cnt >= 0) {
+                    versions = new long[cnt];
+
+                    for (int i = 0; i < cnt; i++)
+                        versions[i] = reader.readLong();
+                }
+
+                IgniteCache<PlatformDotNetEntityFrameworkCacheKey, PlatformDotNetEntityFrameworkCacheEntry> dataCache
+                    = target.rawCache();
+
+                PlatformDotNetEntityFrameworkCacheKey key = new PlatformDotNetEntityFrameworkCacheKey(query, versions);
+
+                PlatformDotNetEntityFrameworkCacheEntry entry = dataCache.get(key);
+
+                byte[] data = entry == null ? null : entry.data();
+
+                return target.writeResult(mem, data);
+            }
+        }
+
+        throw new IgniteCheckedException("Unsupported operation type: " + type);
+    }
+
+    /**
+     * Starts the background cleanup of old cache entries.
+     *
+     * @param grid Grid.
+     * @param metaCache Meta cache.
+     * @param dataCacheName Data cache name.
+     * @param currentVersions Current versions.
+     */
+    private void startBackgroundCleanup(Ignite grid, final Cache<CleanupNodeId, UUID> metaCache,
+        final String dataCacheName, final Map<String, EntryProcessorResult<Long>> currentVersions) {
+        if (cleanupFlags.containsKey(dataCacheName))
+            return;  // Current node already performs cleanup.
+
+        if (!trySetGlobalCleanupFlag(grid, metaCache))
+            return;
+
+        cleanupFlags.put(dataCacheName, true);
+
+        final ClusterGroup dataNodes = grid.cluster().forDataNodes(dataCacheName);
+
+        IgniteCompute asyncCompute = grid.compute(dataNodes).withAsync();
+
+        asyncCompute.broadcast(new RemoveOldEntriesRunnable(dataCacheName, currentVersions));
+
+        asyncCompute.future().listen(new CleanupCompletionListener(metaCache, dataCacheName));
+    }
+
+    /**
+     * Tries to set the global cleanup node id to current node.
+     *
+     * @param grid Grid.
+     * @param metaCache Meta cache.
+     *
+     * @return True if successfully set the flag indicating that current node performs the cleanup; otherwise false.
+     */
+    private boolean trySetGlobalCleanupFlag(Ignite grid, final Cache<CleanupNodeId, UUID> metaCache) {
+        final UUID localNodeId = grid.cluster().localNode().id();
+
+        while (true) {
+            // Get the node performing cleanup.
+            UUID nodeId = metaCache.get(CLEANUP_NODE_ID);
+
+            if (nodeId == null) {
+                if (metaCache.putIfAbsent(CLEANUP_NODE_ID, localNodeId))
+                    return true;  // Successfully reserved cleanup to local node.
+
+                // Failed putIfAbsent: someone else may have started cleanup. Retry the check.
+                continue;
+            }
+
+            if (nodeId.equals(localNodeId))
+                return false;  // Current node already performs cleanup.
+
+            if (grid.cluster().node(nodeId) != null)
+                return false;  // Another node already performs cleanup and is alive.
+
+            // Node that performs cleanup has disconnected.
+            if (metaCache.replace(CLEANUP_NODE_ID, nodeId, localNodeId))
+                return true;  // Successfully replaced disconnected node id with our id.
+
+            // Replace failed: someone else started cleanup.
+            return false;
+        }
+    }
+
+    /**
+     * Removes old cache entries locally.
+     *
+     * @param ignite Ignite.
+     * @param dataCacheName Cache name.
+     * @param currentVersions Current versions.
+     */
+    private static void removeOldEntries(final Ignite ignite, final String dataCacheName,
+        final Map<String, EntryProcessorResult<Long>> currentVersions) {
+
+        IgniteCache<PlatformDotNetEntityFrameworkCacheKey, PlatformDotNetEntityFrameworkCacheEntry> cache =
+            ignite.cache(dataCacheName);
+
+        Set<PlatformDotNetEntityFrameworkCacheKey> keysToRemove = new TreeSet<>();
+
+        ClusterNode localNode = ignite.cluster().localNode();
+
+        for (Cache.Entry<PlatformDotNetEntityFrameworkCacheKey, PlatformDotNetEntityFrameworkCacheEntry> cacheEntry :
+            cache.localEntries(CachePeekMode.ALL)) {
+            // Check if we are on a primary node for the key, since we use CachePeekMode.ALL
+            // and we don't want to process backup entries.
+            if (!ignite.affinity(dataCacheName).isPrimary(localNode, cacheEntry.getKey()))
+                continue;
+
+            long[] versions = cacheEntry.getKey().versions();
+            String[] entitySets = cacheEntry.getValue().entitySets();
+
+            for (int i = 0; i < entitySets.length; i++) {
+                EntryProcessorResult<Long> curVer = currentVersions.get(entitySets[i]);
+
+                if (curVer != null && versions[i] < curVer.get())
+                    keysToRemove.add(cacheEntry.getKey());
+            }
+        }
+
+        cache.removeAll(keysToRemove);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PlatformDotNetEntityFrameworkCacheExtension.class, this);
+    }
+
+    /**
+     * Cache key for cleanup node id.
+     */
+    private static class CleanupNodeId {
+        // No-op.
+    }
+
+    /**
+     * Old entries remover.
+     */
+    private static class RemoveOldEntriesRunnable implements IgniteRunnable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private final String dataCacheName;
+
+        /** */
+        private final Map<String, EntryProcessorResult<Long>> currentVersions;
+
+        /** Inject Ignite. */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /**
+         * Ctor.
+         *
+         * @param dataCacheName Name of the cache to clean up.
+         * @param currentVersions Map of current entity set versions.
+         */
+        private RemoveOldEntriesRunnable(String dataCacheName,
+            Map<String, EntryProcessorResult<Long>> currentVersions) {
+            this.dataCacheName = dataCacheName;
+            this.currentVersions = currentVersions;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            removeOldEntries(ignite, dataCacheName, currentVersions);
+        }
+    }
+
+    /**
+     * Cleanup completion listener.
+     */
+    private class CleanupCompletionListener implements IgniteInClosure<IgniteFuture<Object>> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private final Cache<CleanupNodeId, UUID> metaCache;
+
+        /** */
+        private final String dataCacheName;
+
+        /**
+         * Ctor.
+         *
+         * @param metaCache Metadata cache.
+         * @param dataCacheName Data cache name.
+         */
+        private CleanupCompletionListener(Cache<CleanupNodeId, UUID> metaCache, String dataCacheName) {
+            this.metaCache = metaCache;
+            this.dataCacheName = dataCacheName;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(IgniteFuture<Object> future) {
+            // Reset distributed cleanup flag.
+            metaCache.remove(CLEANUP_NODE_ID);
+
+            // Reset local cleanup flag.
+            cleanupFlags.remove(dataCacheName);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/entityframework/PlatformDotNetEntityFrameworkCacheKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/entityframework/PlatformDotNetEntityFrameworkCacheKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/entityframework/PlatformDotNetEntityFrameworkCacheKey.java
new file mode 100644
index 0000000..60fdaec
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/entityframework/PlatformDotNetEntityFrameworkCacheKey.java
@@ -0,0 +1,164 @@
+/*
+ * 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.platform.entityframework;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryRawReader;
+import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
+import org.jetbrains.annotations.NotNull;
+
+import java.util.Arrays;
+
+/**
+ * EntityFramework cache key: query + versions.
+ */
+@SuppressWarnings("WeakerAccess")
+public class PlatformDotNetEntityFrameworkCacheKey
+    implements Binarylizable, Comparable<PlatformDotNetEntityFrameworkCacheKey> {
+    /** Query text. */
+    private String query;
+
+    /** Entity set versions. */
+    private long[] versions;
+
+    /**
+     * Ctor.
+     */
+    public PlatformDotNetEntityFrameworkCacheKey() {
+        // No-op.
+    }
+
+    /**
+     * Ctor.
+     *
+     * @param query Query text.
+     * @param versions Versions.
+     */
+    PlatformDotNetEntityFrameworkCacheKey(String query, long[] versions) {
+        assert query != null;
+
+        this.query = query;
+        this.versions = versions;
+    }
+
+    /**
+     * Gets the query text.
+     *
+     * @return Query text.
+     */
+    public String query() {
+        return query;
+    }
+
+    /**
+     * Gets the entity set versions.
+     *
+     * @return Entity set versions.
+     */
+    public long[] versions() {
+        return versions;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        PlatformDotNetEntityFrameworkCacheKey key = (PlatformDotNetEntityFrameworkCacheKey)o;
+
+        //noinspection SimplifiableIfStatement
+        if (!query.equals(key.query))
+            return false;
+
+        return Arrays.equals(versions, key.versions);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int result = query.hashCode();
+
+        result = 31 * result + Arrays.hashCode(versions);
+
+        return result;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
+        final BinaryRawWriter raw = writer.rawWriter();
+
+        raw.writeString(query);
+
+        if (versions != null) {
+            raw.writeInt(versions.length);
+
+            for (long ver : versions)
+                raw.writeLong(ver);
+        }
+        else
+            raw.writeInt(-1);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
+        BinaryRawReader raw = reader.rawReader();
+
+        query = raw.readString();
+
+        int cnt = raw.readInt();
+
+        if (cnt >= 0) {
+            versions = new long[cnt];
+
+            for (int i = 0; i < cnt; i++)
+                versions[i] = raw.readLong();
+        }
+        else
+            versions = null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compareTo(@NotNull PlatformDotNetEntityFrameworkCacheKey o) {
+        int cmpQuery = query.compareTo(o.query);
+
+        if (cmpQuery != 0)
+            return cmpQuery;
+
+        if (versions == null) {
+            return o.versions == null ? 0 : -1;
+        }
+
+        if (o.versions == null)
+            return 1;
+
+        assert versions.length == o.versions.length;
+
+        for (int i = 0; i < versions.length; i++) {
+            if (versions[i] != o.versions[i]) {
+                return versions[i] > o.versions[i] ? 1 : -1;
+            }
+        }
+
+        return 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/entityframework/PlatformDotNetEntityFrameworkIncreaseVersionProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/entityframework/PlatformDotNetEntityFrameworkIncreaseVersionProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/entityframework/PlatformDotNetEntityFrameworkIncreaseVersionProcessor.java
new file mode 100644
index 0000000..f10138a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/entityframework/PlatformDotNetEntityFrameworkIncreaseVersionProcessor.java
@@ -0,0 +1,45 @@
+/*
+ * 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.platform.entityframework;
+
+import org.apache.ignite.cache.CacheEntryProcessor;
+
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+
+/**
+ * Entry processor that increments entity set version number.
+ */
+public class PlatformDotNetEntityFrameworkIncreaseVersionProcessor implements CacheEntryProcessor<String, Long, Long> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override public Long process(MutableEntry<String, Long> entry, Object... args) throws EntryProcessorException {
+        Long val = entry.getValue();
+
+        if (val == null)
+            val = 0L;
+
+        val++;
+
+        entry.setValue(val);
+
+        return val;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Apache.Ignite.Core.Tests.NuGet.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Apache.Ignite.Core.Tests.NuGet.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Apache.Ignite.Core.Tests.NuGet.csproj
index a71d1d8..4452ac7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Apache.Ignite.Core.Tests.NuGet.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Apache.Ignite.Core.Tests.NuGet.csproj
@@ -95,6 +95,16 @@
       <HintPath>packages\Apache.Ignite.Log4Net.1.8.0\lib\net40\Apache.Ignite.Log4Net.dll</HintPath>
       <Private>True</Private>
     </Reference>
+    <Reference Include="Apache.Ignite.EntityFramework">
+      <SpecificVersion>False</SpecificVersion>
+      <HintPath>packages\Apache.Ignite.EntityFramework.1.8.0\lib\net40\Apache.Ignite.EntityFramework.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="EntityFramework, Version=6.0.0.0, Culture=neutral, PublicKeyToken=b77a5c561934e089, processorArchitecture=MSIL">
+      <HintPath>packages\EntityFramework.6.1.0\lib\net40\EntityFramework.dll</HintPath>
+      <SpecificVersion>False</SpecificVersion>
+      <Private>True</Private>
+    </Reference>
     <Reference Include="NLog, Version=4.0.0.0, Culture=neutral, PublicKeyToken=5120e14c03d0593c, processorArchitecture=MSIL">
       <HintPath>packages\NLog.4.3.7\lib\net40\NLog.dll</HintPath>
       <Private>True</Private>
@@ -127,6 +137,7 @@
     <Compile Include="AspNetTest.cs" />
     <Compile Include="ComputeTest.cs" />
     <Compile Include="SchemaTest.cs" />
+    <Compile Include="EntityFrameworkCacheTest.cs" />
     <Compile Include="StartupTest.cs" />
     <Compile Include="CacheTest.cs" />
     <Compile Include="Properties\AssemblyInfo.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/EntityFrameworkCacheTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/EntityFrameworkCacheTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/EntityFrameworkCacheTest.cs
new file mode 100644
index 0000000..b4781ce
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/EntityFrameworkCacheTest.cs
@@ -0,0 +1,62 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.NuGet
+{
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.EntityFramework;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests the EntityFramework integration.
+    /// </summary>
+    public class EntityFrameworkCacheTest
+    {
+        /// <summary>
+        /// Tests cache startup and basic operation.
+        /// </summary>
+        [Test]
+        public void TestStartupPutGet()
+        {
+            var cfg = new IgniteConfiguration
+            {
+                DiscoverySpi = TestUtil.GetLocalDiscoverySpi(),
+                GridName = "myGrid"
+            };
+            
+            // ReSharper disable once ObjectCreationAsStatement
+            new IgniteDbConfiguration(cfg,
+                new CacheConfiguration("efMetaCache") {AtomicityMode = CacheAtomicityMode.Transactional},
+                new CacheConfiguration("efDataCache"), null);
+
+            var ignite = Ignition.GetIgnite(cfg.GridName);
+            Assert.IsNotNull(ignite);
+
+            Assert.IsNotNull(ignite.GetCache<string, object>("efMetaCache"));
+            Assert.IsNotNull(ignite.GetCache<string, object>("efDataCache"));
+        }
+
+        /// <summary>
+        /// Test teardown.
+        /// </summary>
+        [TearDown]
+        public void TearDown()
+        {
+            Ignition.StopAll(true);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/packages.config
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/packages.config b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/packages.config
index 80454e0..a7c48f3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/packages.config
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/packages.config
@@ -22,8 +22,10 @@
   <package id="Apache.Ignite.NLog" version="1.8.0" targetFramework="net40" />
   <package id="Apache.Ignite.Log4Net" version="1.8.0" targetFramework="net40" />
   <package id="Apache.Ignite.Schema" version="1.8.0" targetFramework="net40" />
+  <package id="Apache.Ignite.EntityFramework" version="1.8.0" targetFramework="net40" />
   <package id="NLog" version="4.3.7" targetFramework="net40" />
   <package id="NUnit.Runners" version="2.6.3" targetFramework="net40" />
   <package id="Remotion.Linq" version="2.0.1" targetFramework="net40" />
   <package id="log4net" version="2.0.5" targetFramework="net40" />
+  <package id="EntityFramework" version="6.1.0" targetFramework="net40" />
 </packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
index 26c1096..2a2d588 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
@@ -956,10 +956,17 @@ namespace Apache.Ignite.Core.Tests.Cache
             Assert.IsFalse(cache0.ContainsKey(key0));
             Assert.IsFalse(cache0.ContainsKey(key1));
 
+            // Test sliding expiration
             cache0.Put(key0, key0);
             cache0.Put(key1, key1);
-            cache.Get(key0); 
-            cache.Get(key1);
+            for (var i = 0; i < 3; i++)
+            {
+                Thread.Sleep(50);
+
+                // Prolong expiration by touching the entry
+                cache.Get(key0);
+                cache.Get(key1);
+            }
             Assert.IsTrue(cache0.ContainsKey(key0));
             Assert.IsTrue(cache0.ContainsKey(key1));
             Thread.Sleep(200);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
index 4ff3fea..5a9c824 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
@@ -22,6 +22,7 @@ namespace Apache.Ignite.Core.Tests
     using System.Collections.Concurrent;
     using System.Collections.Generic;
     using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
     using System.Linq;
     using System.Threading;
     using Apache.Ignite.Core.Discovery.Tcp;
@@ -346,6 +347,7 @@ namespace Apache.Ignite.Core.Tests
         /// <summary>
         /// Runs the test in new process.
         /// </summary>
+        [SuppressMessage("ReSharper", "AssignNullToNotNullAttribute")]
         public static void RunTestInNewProcess(string fixtureName, string testName)
         {
             var procStart = new ProcessStartInfo

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
index bb5c207..0e5ad2a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
@@ -265,7 +265,7 @@ namespace Apache.Ignite.Core.Impl.Binary
 
                 Debug.Assert(hdr.Version == BinaryUtils.ProtoVer);
                 Debug.Assert(hdr.SchemaOffset <= hdr.Length);
-                Debug.Assert(hdr.SchemaOffset >= Size);
+                Debug.Assert(hdr.SchemaOffset >= Size || !hdr.HasSchema);
 
             }
             else

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Apache.Ignite.EntityFramework.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Apache.Ignite.EntityFramework.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Apache.Ignite.EntityFramework.Tests.csproj
new file mode 100644
index 0000000..9711087
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Apache.Ignite.EntityFramework.Tests.csproj
@@ -0,0 +1,96 @@
+\ufeff<?xml version="1.0" encoding="utf-8"?>
+<Project ToolsVersion="4.0" DefaultTargets="Build" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
+  <Import Project="$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props" Condition="Exists('$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props')" />
+  <PropertyGroup>
+    <Configuration Condition=" '$(Configuration)' == '' ">Debug</Configuration>
+    <Platform Condition=" '$(Platform)' == '' ">AnyCPU</Platform>
+    <ProjectGuid>{CDA5700E-78F3-4A9E-A9B0-704CBE94651C}</ProjectGuid>
+    <OutputType>Library</OutputType>
+    <AppDesignerFolder>Properties</AppDesignerFolder>
+    <RootNamespace>Apache.Ignite.EntityFramework.Tests</RootNamespace>
+    <AssemblyName>Apache.Ignite.EntityFramework.Tests</AssemblyName>
+    <TargetFrameworkVersion>v4.0</TargetFrameworkVersion>
+    <FileAlignment>512</FileAlignment>
+  </PropertyGroup>
+  <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Debug|AnyCPU' ">
+    <DebugSymbols>true</DebugSymbols>
+    <DebugType>full</DebugType>
+    <Optimize>false</Optimize>
+    <OutputPath>bin\Debug\</OutputPath>
+    <DefineConstants>DEBUG;TRACE</DefineConstants>
+    <ErrorReport>prompt</ErrorReport>
+    <WarningLevel>4</WarningLevel>
+  </PropertyGroup>
+  <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release|AnyCPU' ">
+    <DebugType>pdbonly</DebugType>
+    <Optimize>true</Optimize>
+    <OutputPath>bin\Release\</OutputPath>
+    <DefineConstants>TRACE</DefineConstants>
+    <ErrorReport>prompt</ErrorReport>
+    <WarningLevel>4</WarningLevel>
+  </PropertyGroup>
+  <PropertyGroup>
+    <SignAssembly>true</SignAssembly>
+  </PropertyGroup>
+  <PropertyGroup>
+    <AssemblyOriginatorKeyFile>Apache.Ignite.EntityFramework.Tests.snk</AssemblyOriginatorKeyFile>
+  </PropertyGroup>
+  <ItemGroup>
+    <Reference Include="EntityFramework">
+      <HintPath>..\packages\EntityFramework.6.1.3\lib\net40\EntityFramework.dll</HintPath>
+    </Reference>
+    <Reference Include="EntityFramework.SqlServerCompact">
+      <HintPath>..\packages\EntityFramework.SqlServerCompact.6.1.3\lib\net40\EntityFramework.SqlServerCompact.dll</HintPath>
+    </Reference>
+    <Reference Include="nunit.framework">
+      <HintPath>..\packages\NUnit.Runners.2.6.3\tools\nunit.framework.dll</HintPath>
+    </Reference>
+    <Reference Include="System" />
+    <Reference Include="System.Core" />
+    <Reference Include="System.Data.SqlServerCe, Version=4.0.0.0, Culture=neutral, PublicKeyToken=89845dcd8080cc91, processorArchitecture=MSIL">
+      <SpecificVersion>False</SpecificVersion>
+      <HintPath>..\packages\Microsoft.SqlServer.Compact.4.0.8876.1\lib\net40\System.Data.SqlServerCe.dll</HintPath>
+    </Reference>
+    <Reference Include="System.Transactions" />
+    <Reference Include="System.Data.DataSetExtensions" />
+    <Reference Include="System.Data" />
+    <Reference Include="System.Xml" />
+  </ItemGroup>
+  <ItemGroup>
+    <Compile Include="ArrayDbDataReaderTests.cs" />
+    <Compile Include="DbCachingPolicyTest.cs" />
+    <Compile Include="EntityFrameworkCacheInitializationTest.cs" />
+    <Compile Include="EntityFrameworkCacheTest.cs" />
+    <Compile Include="Properties\AssemblyInfo.cs" />
+  </ItemGroup>
+  <ItemGroup>
+    <ProjectReference Include="..\Apache.Ignite.Core.Tests\Apache.Ignite.Core.Tests.csproj">
+      <Project>{6a62f66c-da5b-4fbb-8ce7-a95f740fdc7a}</Project>
+      <Name>Apache.Ignite.Core.Tests</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\Apache.Ignite.Core\Apache.Ignite.Core.csproj">
+      <Project>{4cd2f726-7e2b-46c4-a5ba-057bb82eecb6}</Project>
+      <Name>Apache.Ignite.Core</Name>
+    </ProjectReference>
+    <ProjectReference Include="..\Apache.Ignite.EntityFramework\Apache.Ignite.EntityFramework.csproj">
+      <Project>{c558518a-c1a0-4224-aaa9-a8688474b4dc}</Project>
+      <Name>Apache.Ignite.EntityFramework</Name>
+    </ProjectReference>
+  </ItemGroup>
+  <ItemGroup>
+    <None Include="Apache.Ignite.EntityFramework.Tests.snk" />
+    <None Include="App.config" />
+    <None Include="packages.config">
+      <SubType>Designer</SubType>
+    </None>
+  </ItemGroup>
+  <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
+  <PropertyGroup>
+    <PostBuildEvent>
+		if not exist "$(TargetDir)x86" md "$(TargetDir)x86"
+		xcopy /s /y "$(SolutionDir)packages\Microsoft.SqlServer.Compact.4.0.8876.1\NativeBinaries\x86\*.*" "$(TargetDir)x86"
+		if not exist "$(TargetDir)amd64" md "$(TargetDir)amd64"
+		xcopy /s /y "$(SolutionDir)packages\Microsoft.SqlServer.Compact.4.0.8876.1\NativeBinaries\amd64\*.*" "$(TargetDir)amd64"
+	</PostBuildEvent>
+  </PropertyGroup>
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Apache.Ignite.EntityFramework.Tests.snk
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Apache.Ignite.EntityFramework.Tests.snk b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Apache.Ignite.EntityFramework.Tests.snk
new file mode 100644
index 0000000..5ef85a6
Binary files /dev/null and b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Apache.Ignite.EntityFramework.Tests.snk differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/App.config
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/App.config b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/App.config
new file mode 100644
index 0000000..3527920
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/App.config
@@ -0,0 +1,71 @@
+\ufeff<?xml version="1.0" encoding="utf-8" ?>
+<!--
+  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.
+-->
+<configuration>
+    <configSections>
+        <section name="igniteConfiguration" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
+        <section name="igniteConfiguration2" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
+        <section name="entityFramework" type="System.Data.Entity.Internal.ConfigFile.EntityFrameworkSection, EntityFramework, Version=6.0.0.0, Culture=neutral, PublicKeyToken=b77a5c561934e089" requirePermission="false" />
+    </configSections>
+    <runtime>
+        <gcServer enabled="true" />
+    </runtime>
+
+    <igniteConfiguration xmlns="http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection" gridName="myGrid1">
+        <discoverySpi type="TcpDiscoverySpi">
+            <ipFinder type="TcpDiscoveryStaticIpFinder">
+                <endpoints>
+                    <string>127.0.0.1:47500</string>
+                </endpoints>
+            </ipFinder>
+        </discoverySpi>
+        <cacheConfiguration>
+            <cacheConfiguration name="cacheName" />
+        </cacheConfiguration>
+    </igniteConfiguration>
+
+    <igniteConfiguration2 gridName="myGrid2" localhost="127.0.0.1">
+        <discoverySpi type="TcpDiscoverySpi">
+            <ipFinder type="TcpDiscoveryStaticIpFinder">
+                <endpoints>
+                    <string>127.0.0.1:47500</string>
+                </endpoints>
+            </ipFinder>
+        </discoverySpi>
+        <cacheConfiguration>
+            <cacheConfiguration name="cacheName2" atomicityMode="Transactional" />
+        </cacheConfiguration>
+    </igniteConfiguration2>
+
+    <entityFramework>
+        <defaultConnectionFactory type="System.Data.Entity.Infrastructure.SqlCeConnectionFactory, EntityFramework">
+            <parameters>
+                <parameter value="System.Data.SqlServerCe.4.0" />
+            </parameters>
+        </defaultConnectionFactory>
+        <providers>
+            <provider invariantName="System.Data.SqlServerCe.4.0" type="System.Data.Entity.SqlServerCompact.SqlCeProviderServices, EntityFramework.SqlServerCompact" />
+        </providers>
+    </entityFramework>
+
+    <system.data>
+        <DbProviderFactories>
+            <remove invariant="System.Data.SqlServerCe.4.0" />
+            <add name="Microsoft SQL Server Compact Data Provider 4.0" invariant="System.Data.SqlServerCe.4.0" description=".NET Framework Data Provider for Microsoft SQL Server Compact" type="System.Data.SqlServerCe.SqlCeProviderFactory, System.Data.SqlServerCe, Version=4.0.0.0, Culture=neutral, PublicKeyToken=89845dcd8080cc91" />
+        </DbProviderFactories>
+    </system.data>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/ArrayDbDataReaderTests.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/ArrayDbDataReaderTests.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/ArrayDbDataReaderTests.cs
new file mode 100644
index 0000000..f67fed4
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/ArrayDbDataReaderTests.cs
@@ -0,0 +1,192 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.EntityFramework.Tests
+{
+    using System;
+    using System.Linq;
+    using Apache.Ignite.EntityFramework.Impl;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests for <see cref="ArrayDbDataReader"/>.
+    /// </summary>
+    public class ArrayDbDataReaderTests
+    {
+        /// <summary>
+        /// Tests the reader.
+        /// </summary>
+        [Test]
+        public void TestReader()
+        {
+            var dateTime = DateTime.Now;
+            var guid = Guid.NewGuid();
+
+            var data = new[]
+            {
+                new object[]
+                {
+                    (byte) 1, (short) 2, 3, (long) 4, (float) 5, (double) 6, (decimal) 7, "8", '9', dateTime,
+                    guid, false, new byte[] {1,2}, new[] {'a','b'}
+                }
+            };
+
+            var schema = new []
+            {
+                new DataReaderField("fbyte", typeof(byte), "by"),
+                new DataReaderField("fshort", typeof(short), "sh"),
+                new DataReaderField("fint", typeof(int), "in"),
+                new DataReaderField("flong", typeof(long), "lo"),
+                new DataReaderField("ffloat", typeof(float), "fl"),
+                new DataReaderField("fdouble", typeof(double), "do"),
+                new DataReaderField("fdecimal", typeof(decimal), "de"),
+                new DataReaderField("fstring", typeof(string), "st"),
+                new DataReaderField("fchar", typeof(char), "ch"),
+                new DataReaderField("fDateTime", typeof(DateTime), "Da"),
+                new DataReaderField("fGuid", typeof(Guid), "Gu"),
+                new DataReaderField("fbool", typeof(bool), "bo"),
+                new DataReaderField("fbytes", typeof(byte[]), "bb"),
+                new DataReaderField("fchars", typeof(char[]), "cc"),
+            };
+
+            // Create reader,
+            var reader = new ArrayDbDataReader(data, schema);
+
+            // Check basic props.
+            Assert.IsTrue(reader.Read());
+            Assert.AreEqual(0, reader.Depth);
+            Assert.AreEqual(-1, reader.RecordsAffected);
+            Assert.AreEqual(14, reader.FieldCount);
+            Assert.AreEqual(14, reader.VisibleFieldCount);
+            Assert.IsFalse(reader.IsClosed);
+            Assert.IsTrue(reader.HasRows);
+
+            // Check reading.
+            var data2 = new object[14];
+            Assert.AreEqual(14, reader.GetValues(data2));
+            Assert.AreEqual(data[0], data2);
+
+            Assert.AreEqual(1, reader.GetByte(reader.GetOrdinal("fbyte")));
+            Assert.AreEqual("by", reader.GetDataTypeName(0));
+            Assert.AreEqual(typeof(byte), reader.GetFieldType(0));
+            Assert.AreEqual("fbyte", reader.GetName(0));
+            Assert.AreEqual(1, reader["fbyte"]);
+            Assert.AreEqual(1, reader[0]);
+
+            Assert.AreEqual(2, reader.GetInt16(reader.GetOrdinal("fshort")));
+            Assert.AreEqual("sh", reader.GetDataTypeName(1));
+            Assert.AreEqual(typeof(short), reader.GetFieldType(1));
+            Assert.AreEqual("fshort", reader.GetName(1));
+            Assert.AreEqual(2, reader["fshort"]);
+            Assert.AreEqual(2, reader[1]);
+
+            Assert.AreEqual(3, reader.GetInt32(reader.GetOrdinal("fint")));
+            Assert.AreEqual("in", reader.GetDataTypeName(2));
+            Assert.AreEqual(typeof(int), reader.GetFieldType(2));
+            Assert.AreEqual("fint", reader.GetName(2));
+            Assert.AreEqual(3, reader["fint"]);
+            Assert.AreEqual(3, reader[2]);
+
+            Assert.AreEqual(4, reader.GetInt64(reader.GetOrdinal("flong")));
+            Assert.AreEqual("lo", reader.GetDataTypeName(3));
+            Assert.AreEqual(typeof(long), reader.GetFieldType(3));
+            Assert.AreEqual("flong", reader.GetName(3));
+            Assert.AreEqual(4, reader["flong"]);
+            Assert.AreEqual(4, reader[3]);
+
+            Assert.AreEqual(5, reader.GetFloat(reader.GetOrdinal("ffloat")));
+            Assert.AreEqual("fl", reader.GetDataTypeName(4));
+            Assert.AreEqual(typeof(float), reader.GetFieldType(4));
+            Assert.AreEqual("ffloat", reader.GetName(4));
+            Assert.AreEqual(5, reader["ffloat"]);
+            Assert.AreEqual(5, reader[4]);
+
+            Assert.AreEqual(6, reader.GetDouble(reader.GetOrdinal("fdouble")));
+            Assert.AreEqual("do", reader.GetDataTypeName(5));
+            Assert.AreEqual(typeof(double), reader.GetFieldType(5));
+            Assert.AreEqual("fdouble", reader.GetName(5));
+            Assert.AreEqual(6, reader["fdouble"]);
+            Assert.AreEqual(6, reader[5]);
+
+            Assert.AreEqual(7, reader.GetDecimal(reader.GetOrdinal("fdecimal")));
+            Assert.AreEqual("de", reader.GetDataTypeName(6));
+            Assert.AreEqual(typeof(decimal), reader.GetFieldType(6));
+            Assert.AreEqual("fdecimal", reader.GetName(6));
+            Assert.AreEqual(7, reader["fdecimal"]);
+            Assert.AreEqual(7, reader[6]);
+
+            Assert.AreEqual("8", reader.GetString(reader.GetOrdinal("fstring")));
+            Assert.AreEqual("st", reader.GetDataTypeName(7));
+            Assert.AreEqual(typeof(string), reader.GetFieldType(7));
+            Assert.AreEqual("fstring", reader.GetName(7));
+            Assert.AreEqual("8", reader["fstring"]);
+            Assert.AreEqual("8", reader[7]);
+
+            Assert.AreEqual('9', reader.GetChar(reader.GetOrdinal("fchar")));
+            Assert.AreEqual("ch", reader.GetDataTypeName(8));
+            Assert.AreEqual(typeof(char), reader.GetFieldType(8));
+            Assert.AreEqual("fchar", reader.GetName(8));
+            Assert.AreEqual('9', reader["fchar"]);
+            Assert.AreEqual('9', reader[8]);
+
+            Assert.AreEqual(dateTime, reader.GetDateTime(reader.GetOrdinal("fDateTime")));
+            Assert.AreEqual("Da", reader.GetDataTypeName(9));
+            Assert.AreEqual(typeof(DateTime), reader.GetFieldType(9));
+            Assert.AreEqual("fDateTime", reader.GetName(9));
+            Assert.AreEqual(dateTime, reader["fDateTime"]);
+            Assert.AreEqual(dateTime, reader[9]);
+
+            Assert.AreEqual(guid, reader.GetGuid(reader.GetOrdinal("fGuid")));
+            Assert.AreEqual("Gu", reader.GetDataTypeName(10));
+            Assert.AreEqual(typeof(Guid), reader.GetFieldType(10));
+            Assert.AreEqual("fGuid", reader.GetName(10));
+            Assert.AreEqual(guid, reader["fGuid"]);
+            Assert.AreEqual(guid, reader[10]);
+
+            Assert.AreEqual(false, reader.GetBoolean(reader.GetOrdinal("fbool")));
+            Assert.AreEqual("bo", reader.GetDataTypeName(11));
+            Assert.AreEqual(typeof(bool), reader.GetFieldType(11));
+            Assert.AreEqual("fbool", reader.GetName(11));
+            Assert.AreEqual(false, reader["fbool"]);
+            Assert.AreEqual(false, reader[11]);
+
+            var bytes = new byte[2];
+            Assert.AreEqual(2, reader.GetBytes(reader.GetOrdinal("fbytes"),0, bytes, 0, 2));
+            Assert.AreEqual(data[0][12], bytes);
+            Assert.AreEqual("bb", reader.GetDataTypeName(12));
+            Assert.AreEqual(typeof(byte[]), reader.GetFieldType(12));
+            Assert.AreEqual("fbytes", reader.GetName(12));
+            Assert.AreEqual(data[0][12], reader["fbytes"]);
+            Assert.AreEqual(data[0][12], reader[12]);
+
+            var chars = new char[2];
+            Assert.AreEqual(2, reader.GetChars(reader.GetOrdinal("fchars"),0, chars, 0, 2));
+            Assert.AreEqual(data[0][13], chars);
+            Assert.AreEqual("cc", reader.GetDataTypeName(13));
+            Assert.AreEqual(typeof(char[]), reader.GetFieldType(13));
+            Assert.AreEqual("fchars", reader.GetName(13));
+            Assert.AreEqual(data[0][13], reader["fchars"]);
+            Assert.AreEqual(data[0][13], reader[13]);
+
+            Assert.IsFalse(Enumerable.Range(0, 14).Any(x => reader.IsDBNull(x)));
+
+            // Close.
+            reader.Close();
+            Assert.IsTrue(reader.IsClosed);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/DbCachingPolicyTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/DbCachingPolicyTest.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/DbCachingPolicyTest.cs
new file mode 100644
index 0000000..c9456b6
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/DbCachingPolicyTest.cs
@@ -0,0 +1,43 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.EntityFramework.Tests
+{
+    using System;
+    using Apache.Ignite.EntityFramework;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests for <see cref="DbCachingPolicy"/>.
+    /// </summary>
+    public class DbCachingPolicyTest
+    {
+        /// <summary>
+        /// Tests the default implementation.
+        /// </summary>
+        [Test]
+        public void TestDefaultImpl()
+        {
+            var plc = new DbCachingPolicy();
+
+            Assert.IsTrue(plc.CanBeCached(null));
+            Assert.IsTrue(plc.CanBeCached(null, 0));
+            Assert.AreEqual(TimeSpan.MaxValue, plc.GetExpirationTimeout(null));
+            Assert.AreEqual(DbCachingMode.ReadWrite, plc.GetCachingMode(null));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheInitializationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheInitializationTest.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheInitializationTest.cs
new file mode 100644
index 0000000..36b1c2b
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheInitializationTest.cs
@@ -0,0 +1,137 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.EntityFramework.Tests
+{
+    using System;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Tests;
+    using Apache.Ignite.EntityFramework;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests the EF cache provider.
+    /// </summary>
+    public class EntityFrameworkCacheInitializationTest
+    {
+        /// <summary>
+        /// Fixture tear down.
+        /// </summary>
+        [TestFixtureTearDown]
+        public void TestFixtureTearDown()
+        {
+            Ignition.StopAll(true);
+        }
+
+        /// <summary>
+        /// Tests the IgniteDbConfiguration.
+        /// </summary>
+        [Test]
+        public void TestConfigurationAndStartup()
+        {
+            Environment.SetEnvironmentVariable("IGNITE_NATIVE_TEST_CLASSPATH", "true");
+
+            Assert.IsNull(Ignition.TryGetIgnite());
+
+            // Test default config (picks up app.config section).
+            CheckCacheAndStop("myGrid1", IgniteDbConfiguration.DefaultCacheNamePrefix, new IgniteDbConfiguration());
+
+            // Specific config section.
+            CheckCacheAndStop("myGrid2", "cacheName2",
+                new IgniteDbConfiguration("igniteConfiguration2", "cacheName2", null));
+
+            // Specific config section, nonexistent cache.
+            CheckCacheAndStop("myGrid2", "newCache",
+                new IgniteDbConfiguration("igniteConfiguration2", "newCache", null));
+
+            // In-code configuration.
+            CheckCacheAndStop("myGrid3", "myCache",
+                new IgniteDbConfiguration(new IgniteConfiguration
+                    {
+                        GridName = "myGrid3",
+                    }, new CacheConfiguration("myCache_metadata")
+                    {
+                        CacheMode = CacheMode.Replicated,
+                        AtomicityMode = CacheAtomicityMode.Transactional
+                    },
+                    new CacheConfiguration("myCache_data") {CacheMode = CacheMode.Replicated}, null),
+                CacheMode.Replicated);
+
+            // Existing instance.
+            var ignite = Ignition.Start(TestUtils.GetTestConfiguration());
+            CheckCacheAndStop(null, "123", new IgniteDbConfiguration(ignite,
+                new CacheConfiguration("123_metadata")
+                {
+                    Backups = 1,
+                    AtomicityMode = CacheAtomicityMode.Transactional
+                },
+                new CacheConfiguration("123_data"), null));
+
+            // Non-tx meta cache.
+            var ex = Assert.Throws<IgniteException>(() => CheckCacheAndStop(null, "123",
+                new IgniteDbConfiguration(TestUtils.GetTestConfiguration(), 
+                    new CacheConfiguration("123_metadata"),
+                    new CacheConfiguration("123_data"), null)));
+
+            Assert.AreEqual("EntityFramework meta cache should be Transactional.", ex.Message);
+
+            // Same cache names.
+            var ex2 = Assert.Throws<ArgumentException>(() => CheckCacheAndStop(null, "abc",
+                new IgniteDbConfiguration(TestUtils.GetTestConfiguration(),
+                    new CacheConfiguration("abc"),
+                    new CacheConfiguration("abc"), null)));
+
+            Assert.IsTrue(ex2.Message.Contains("Meta and Data cache can't have the same name."));
+        }
+
+        /// <summary>
+        /// Checks that specified cache exists and stops all Ignite instances.
+        /// </summary>
+        // ReSharper disable once UnusedParameter.Local
+        private static void CheckCacheAndStop(string gridName, string cacheName, IgniteDbConfiguration cfg,
+            CacheMode cacheMode = CacheMode.Partitioned)
+        {
+            try
+            {
+                Assert.IsNotNull(cfg);
+
+                var ignite = Ignition.TryGetIgnite(gridName);
+                Assert.IsNotNull(ignite);
+
+                var metaCache = ignite.GetCache<object, object>(cacheName + "_metadata");
+                Assert.IsNotNull(metaCache);
+                Assert.AreEqual(cacheMode, metaCache.GetConfiguration().CacheMode);
+
+                if (cacheMode == CacheMode.Partitioned)
+                    Assert.AreEqual(1, metaCache.GetConfiguration().Backups);
+
+                var dataCache = ignite.GetCache<object, object>(cacheName + "_data");
+                Assert.IsNotNull(dataCache);
+                Assert.AreEqual(cacheMode, dataCache.GetConfiguration().CacheMode);
+
+                if (cacheMode == CacheMode.Partitioned)
+                    Assert.AreEqual(0, dataCache.GetConfiguration().Backups);
+            }
+            finally
+            {
+                Ignition.StopAll(true);
+            }
+        }
+    }
+}


[25/50] [abbrv] ignite git commit: .NET: Fix minor analysis warnings

Posted by vo...@apache.org.
.NET: Fix minor analysis warnings


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

Branch: refs/heads/ignite-2693
Commit: 1612b6d66fed032182a41e90da71e6b986ae087b
Parents: 319014d
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Nov 8 14:07:54 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Nov 8 14:07:54 2016 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs   | 2 +-
 .../dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs    | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1612b6d6/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs
index f6a2f07..86b1406 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs
@@ -62,7 +62,7 @@ namespace Apache.Ignite.Core.Compute
         /// </returns>
         /// <exception cref="IgniteException">Split returned no jobs.</exception>
         [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
-        override public IDictionary<IComputeJob<TJobRes>, IClusterNode> Map(IList<IClusterNode> subgrid, TArg arg)
+        public override IDictionary<IComputeJob<TJobRes>, IClusterNode> Map(IList<IClusterNode> subgrid, TArg arg)
         {
             var jobs = Split(subgrid.Count, arg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1612b6d6/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs
index e54a199..a5e2cb0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs
@@ -24,7 +24,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
     /// <summary>
     /// Base unmanaged target implementation.
     /// </summary>
-    internal unsafe sealed class UnmanagedTarget : CriticalHandle, IUnmanagedTarget
+    internal sealed unsafe class UnmanagedTarget : CriticalHandle, IUnmanagedTarget
     {
         /** Context. */
         private readonly UnmanagedContext _ctx;


[04/50] [abbrv] ignite git commit: IGNITE-4117 .NET: Fix ClientReconnectTask completion timing

Posted by vo...@apache.org.
IGNITE-4117 .NET: Fix ClientReconnectTask completion timing

This closes #1195


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

Branch: refs/heads/ignite-2693
Commit: 4495efc2aa37ca31214558968fa83598317e830a
Parents: e7d7a59
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Nov 1 13:22:30 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Nov 1 13:22:30 2016 +0300

----------------------------------------------------------------------
 .../platform/PlatformProcessorImpl.java         | 15 +++-
 .../Apache.Ignite.Core.Tests/ReconnectTest.cs   | 81 ++++++++++++++++++--
 2 files changed, 88 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4495efc2/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
index 548145e..d875c7e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
@@ -54,6 +54,7 @@ import org.apache.ignite.internal.processors.platform.services.PlatformServices;
 import org.apache.ignite.internal.processors.platform.transactions.PlatformTransactions;
 import org.apache.ignite.internal.processors.platform.utils.PlatformConfigurationUtils;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
+import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteFuture;
@@ -101,6 +102,9 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     /** Cache extensions. */
     private final PlatformCacheExtension[] cacheExts;
 
+    /** Cluster restart flag for the reconnect callback. */
+    private volatile boolean clusterRestarted;
+
     /**
      * Constructor.
      *
@@ -378,11 +382,20 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     /** {@inheritDoc} */
     @Override public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
         platformCtx.gateway().onClientDisconnected();
+
+        // 1) onReconnected is called on all grid components.
+        // 2) After all of grid components have completed their reconnection, reconnectFut is completed.
+        reconnectFut.listen(new CI1<IgniteFuture<?>>() {
+            @Override public void apply(IgniteFuture<?> future) {
+                platformCtx.gateway().onClientReconnected(clusterRestarted);
+            }
+        });
     }
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
-        platformCtx.gateway().onClientReconnected(clusterRestarted);
+        // Save the flag value for callback of reconnectFut.
+        this.clusterRestarted = clusterRestarted;
 
         return null;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4495efc2/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs
index 35bbca5..91e4c06 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs
@@ -17,8 +17,12 @@
 
 namespace Apache.Ignite.Core.Tests
 {
+    using System;
+    using System.Threading;
     using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Lifecycle;
     using Apache.Ignite.Core.Tests.Process;
     using NUnit.Framework;
 
@@ -28,11 +32,71 @@ namespace Apache.Ignite.Core.Tests
     [Category(TestUtils.CategoryIntensive)]
     public class ReconnectTest
     {
+        /** */
+        private const string CacheName = "cache";
+
         /// <summary>
-        /// Tests the disconnected exception.
+        /// Tests the cluster restart scenario, where client is alive, but all servers restart.
         /// </summary>
         [Test]
-        public void TestDisconnectedException()
+        public void TestClusterRestart()
+        {
+            var serverCfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                CacheConfiguration = new[] {new CacheConfiguration(CacheName)}
+            };
+
+            var clientCfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                GridName = "client",
+                ClientMode = true
+            };
+
+            var server = Ignition.Start(serverCfg);
+            var client = Ignition.Start(clientCfg);
+
+            ClientReconnectEventArgs eventArgs = null;
+
+            client.ClientReconnected += (sender, args) => { eventArgs = args; };
+
+            var cache = client.GetCache<int, int>(CacheName);
+
+            cache[1] = 1;
+
+            Ignition.Stop(server.Name, true);
+
+            var cacheEx = Assert.Throws<CacheException>(() => cache.Get(1));
+            var ex = cacheEx.InnerException as ClientDisconnectedException;
+
+            Assert.IsNotNull(ex);
+
+            // Start the server and wait for reconnect.
+            Ignition.Start(serverCfg);
+            Assert.IsTrue(ex.ClientReconnectTask.Result);
+
+            // Check the event args.
+            Thread.Sleep(1);  // Wait for event handler
+
+            Assert.IsNotNull(eventArgs);
+            Assert.IsTrue(eventArgs.HasClusterRestarted);
+
+            // Refresh the cache instance and check that it works.
+            var cache1 = client.GetCache<int, int>(CacheName);
+            Assert.AreEqual(0, cache1.GetSize());
+
+            cache1[1] = 2;
+            Assert.AreEqual(2, cache1[1]);
+
+            // Check that old cache instance does not work.
+            var cacheEx1 = Assert.Throws<InvalidOperationException>(() => cache.Get(1));
+            Assert.AreEqual("Cache has been closed or destroyed: " + CacheName, cacheEx1.Message);
+        }
+
+        /// <summary>
+        /// Tests the failed connection scenario, where servers are alive, but can't be contacted.
+        /// </summary>
+        [Test]
+        public void TestFailedConnection()
         {
             var cfg = new IgniteConfiguration
             {
@@ -54,7 +118,7 @@ namespace Apache.Ignite.Core.Tests
 
                 Assert.IsTrue(ignite.GetCluster().ClientReconnectTask.IsCompleted);
 
-                var cache = ignite.CreateCache<int, int>("c");
+                var cache = ignite.CreateCache<int, int>(CacheName);
 
                 cache[1] = 1;
 
@@ -69,6 +133,8 @@ namespace Apache.Ignite.Core.Tests
 
                 var inner = (ClientDisconnectedException) ex.InnerException;
 
+                Assert.IsNotNull(inner);
+
                 var clientReconnectTask = inner.ClientReconnectTask;
 
                 Assert.AreEqual(ignite.GetCluster().ClientReconnectTask, clientReconnectTask);
@@ -78,7 +144,7 @@ namespace Apache.Ignite.Core.Tests
                 // Resume process to reconnect
                 proc.Resume();
 
-                clientReconnectTask.Wait();
+                Assert.IsFalse(clientReconnectTask.Result);
 
                 Assert.AreEqual(1, cache[1]);
                 Assert.AreEqual(1, disconnected);
@@ -97,11 +163,12 @@ namespace Apache.Ignite.Core.Tests
         }
 
         /// <summary>
-        /// Fixture tear down.
+        /// Test tear down.
         /// </summary>
-        [TestFixtureTearDown]
-        public void FixtureTearDown()
+        [TearDown]
+        public void TearDown()
         {
+            Ignition.StopAll(true);
             IgniteProcess.KillAll();
             Ignition.ClientMode = false;
         }


[11/50] [abbrv] ignite git commit: IGNITE-3630 .NET: Add pure binary mode example with put-get and queries.

Posted by vo...@apache.org.
IGNITE-3630 .NET: Add pure binary mode example with put-get and queries.


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

Branch: refs/heads/ignite-2693
Commit: 3a9cbed4e692e76ad884e60a4f5edc670e6b3c8f
Parents: 861c173
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Aug 8 17:00:04 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Wed Nov 2 11:24:25 2016 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Examples.csproj               |   1 +
 .../Datagrid/BinaryModeExample.cs               | 272 +++++++++++++++++++
 2 files changed, 273 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3a9cbed4/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
index ab0a9f2..9b1d323 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
@@ -54,6 +54,7 @@
     <Compile Include="Datagrid\DataStreamerExample.cs" />
     <Compile Include="Datagrid\PutGetExample.cs" />
     <Compile Include="Datagrid\LinqExample.cs" />
+    <Compile Include="Datagrid\BinaryModeExample.cs" />
     <Compile Include="Datagrid\QueryExample.cs" />
     <Compile Include="Datagrid\StoreExample.cs" />
     <Compile Include="Datagrid\TransactionExample.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/3a9cbed4/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/BinaryModeExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/BinaryModeExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/BinaryModeExample.cs
new file mode 100644
index 0000000..1c430b3
--- /dev/null
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/BinaryModeExample.cs
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Examples.Datagrid
+{
+    using System;
+    using System.Collections;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Cache.Query;
+
+    /// <summary>
+    /// This example works with cache entirely in binary mode: no classes or configurations are needed.
+    /// <para />
+    /// 1) Set this class as startup object (Apache.Ignite.Examples project -> right-click -> Properties ->
+    ///     Application -> Startup object);
+    /// 2) Start example (F5 or Ctrl+F5).
+    /// <para />
+    /// This example can be run with standalone Apache Ignite.NET node:
+    /// 1) Run %IGNITE_HOME%/platforms/dotnet/bin/Apache.Ignite.exe:
+    /// Apache.Ignite.exe -configFileName=platforms\dotnet\examples\apache.ignite.examples\app.config
+    /// 2) Start example.
+    /// </summary>
+    public class BinaryModeExample
+    {
+        /// <summary>Cache name.</summary>
+        private const string CacheName = "dotnet_binary_cache";
+
+        /// <summary>Person type name.</summary>
+        private const string PersonType = "Person";
+
+        /// <summary>Company type name.</summary>
+        private const string CompanyType = "Company";
+
+        /// <summary>Name field name.</summary>
+        private const string NameField = "Name";
+        
+        /// <summary>Company ID field name.</summary>
+        private const string CompanyIdField = "CompanyId";
+
+        /// <summary>ID field name.</summary>
+        private const string IdField = "Id";
+
+        [STAThread]
+        public static void Main()
+        {
+            using (var ignite = Ignition.StartFromApplicationConfiguration())
+            {
+                Console.WriteLine();
+                Console.WriteLine(">>> Binary mode example started.");
+
+                // Create new cache and configure queries for Person and Company binary types.
+                // Note that there are no such classes defined.
+                var cache0 = ignite.GetOrCreateCache<object, object>(new CacheConfiguration
+                {
+                    Name = CacheName,
+                    QueryEntities = new[]
+                    {
+                        new QueryEntity
+                        {
+                            KeyType = typeof(int),
+                            ValueTypeName = PersonType,
+                            Fields = new[]
+                            {
+                                new QueryField(NameField, typeof(string)),
+                                new QueryField(CompanyIdField, typeof(int))
+                            },
+                            Indexes = new[] {new QueryIndex(false, QueryIndexType.FullText, NameField)}
+                        },
+                        new QueryEntity
+                        {
+                            KeyType = typeof(int),
+                            ValueTypeName = CompanyType,
+                            Fields = new[]
+                            {
+                                new QueryField(IdField, typeof(int)),
+                                new QueryField(NameField, typeof(string))
+                            }
+                        }
+                    }
+                });
+
+                // Switch to binary mode to work with data in serialized form.
+                var cache = cache0.WithKeepBinary<int, IBinaryObject>();
+
+                // Clean up caches on all nodes before run.
+                cache.Clear();
+
+                // Populate cache with sample data entries.
+                PopulateCache(cache);
+
+                // Run read & modify example.
+                ReadModifyExample(cache);
+
+                // Run SQL query example.
+                SqlQueryExample(cache);
+
+                // Run SQL query with join example.
+                SqlJoinQueryExample(cache);
+
+                // Run SQL fields query example.
+                SqlFieldsQueryExample(cache);
+
+                // Run full text query example.
+                FullTextQueryExample(cache);
+
+                Console.WriteLine();
+            }
+
+            Console.WriteLine();
+            Console.WriteLine(">>> Example finished, press any key to exit ...");
+            Console.ReadKey();
+        }
+
+        /// <summary>
+        /// Reads binary object fields and modifies them.
+        /// </summary>
+        /// <param name="cache">Cache.</param>
+        private static void ReadModifyExample(ICache<int, IBinaryObject> cache)
+        {
+            const int id = 1;
+
+            IBinaryObject person = cache[id];
+
+            string name = person.GetField<string>(NameField);
+
+            Console.WriteLine();
+            Console.WriteLine(">>> Name of the person with id {0}: {1}", id, name);
+
+            // Modify the binary object.
+            cache[id] = person.ToBuilder().SetField("Name", name + " Jr.").Build();
+
+            Console.WriteLine(">>> Modified person with id {0}: {1}", id, cache[1]);
+        }
+
+        /// <summary>
+        /// Queries persons that have a specific name using SQL.
+        /// </summary>
+        /// <param name="cache">Cache.</param>
+        private static void SqlQueryExample(ICache<int, IBinaryObject> cache)
+        {
+            var qry = cache.Query(new SqlQuery(PersonType, "name like 'James%'"));
+
+            Console.WriteLine();
+            Console.WriteLine(">>> Persons named James:");
+
+            foreach (var entry in qry)
+                Console.WriteLine(">>>    " + entry.Value);
+        }
+
+        /// <summary>
+        /// Queries persons that work for company with provided name.
+        /// </summary>
+        /// <param name="cache">Cache.</param>
+        private static void SqlJoinQueryExample(ICache<int, IBinaryObject> cache)
+        {
+            const string orgName = "Apache";
+
+            var qry = cache.Query(new SqlQuery(PersonType,
+                "from Person, Company " +
+                "where Person.CompanyId = Company.Id and Company.Name = ?", orgName));
+
+            Console.WriteLine();
+            Console.WriteLine(">>> Persons working for " + orgName + ":");
+
+            foreach (var entry in qry)
+                Console.WriteLine(">>>     " + entry.Value);
+        }
+
+        /// <summary>
+        /// Queries names for all persons.
+        /// </summary>
+        /// <param name="cache">Cache.</param>
+        private static void SqlFieldsQueryExample(ICache<int, IBinaryObject> cache)
+        {
+            var qry = cache.QueryFields(new SqlFieldsQuery("select name from Person order by name"));
+
+            Console.WriteLine();
+            Console.WriteLine(">>> All person names:");
+
+            foreach (IList row in qry)
+                Console.WriteLine(">>>     " + row[0]);
+        }
+
+        /// <summary>
+        /// Queries persons that have a specific name using full-text query API.
+        /// </summary>
+        /// <param name="cache">Cache.</param>
+        private static void FullTextQueryExample(ICache<int, IBinaryObject> cache)
+        {
+            var qry = cache.Query(new TextQuery(PersonType, "Peters"));
+
+            Console.WriteLine();
+            Console.WriteLine(">>> Persons named Peters:");
+
+            foreach (var entry in qry)
+                Console.WriteLine(">>>     " + entry.Value);
+        }
+
+        /// <summary>
+        /// Populate cache with data for this example.
+        /// </summary>
+        /// <param name="cache">Cache.</param>
+        private static void PopulateCache(ICache<int, IBinaryObject> cache)
+        {
+            IBinary binary = cache.Ignite.GetBinary();
+
+            // Populate persons.
+            cache[1] = binary.GetBuilder(PersonType)
+                .SetField(NameField, "James Wilson")
+                .SetField(CompanyIdField, -1)
+                .Build();
+
+            cache[2] = binary.GetBuilder(PersonType)
+                .SetField(NameField, "Daniel Adams")
+                .SetField(CompanyIdField, -1)
+                .Build();
+
+            cache[3] = binary.GetBuilder(PersonType)
+                .SetField(NameField, "Cristian Moss")
+                .SetField(CompanyIdField, -1)
+                .Build();
+
+            cache[4] = binary.GetBuilder(PersonType)
+                .SetField(NameField, "Allison Mathis")
+                .SetField(CompanyIdField, -2)
+                .Build();
+
+            cache[5] = binary.GetBuilder(PersonType)
+                .SetField(NameField, "Breana Robbin")
+                .SetField(CompanyIdField, -2)
+                .Build();
+
+            cache[6] = binary.GetBuilder(PersonType)
+                .SetField(NameField, "Philip Horsley")
+                .SetField(CompanyIdField, -2)
+                .Build();
+
+            cache[7] = binary.GetBuilder(PersonType)
+                .SetField(NameField, "James Peters")
+                .SetField(CompanyIdField, -2)
+                .Build();
+
+            // Populate companies.
+            cache[-1] = binary.GetBuilder(CompanyType)
+                .SetField(NameField, "Apache")
+                .SetField(IdField, -1)
+                .Build();
+
+            cache[-2] = binary.GetBuilder(CompanyType)
+                .SetField(NameField, "Microsoft")
+                .SetField(IdField, -2)
+                .Build();
+        }
+    }
+}


[05/50] [abbrv] ignite git commit: IGNITE-3368 .NET: Improve test coverage

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/IgniteExceptionTaskSelfTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/IgniteExceptionTaskSelfTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/IgniteExceptionTaskSelfTest.cs
index 0c983fd..912102c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/IgniteExceptionTaskSelfTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/IgniteExceptionTaskSelfTest.cs
@@ -105,7 +105,7 @@ namespace Apache.Ignite.Core.Tests.Compute
 
             Assert.AreEqual(1, res);
 
-            Assert.AreEqual(1, JobErrs.Count);
+            Assert.AreEqual(4, JobErrs.Count);
             Assert.IsNotNull(JobErrs.First() as GoodException);
             Assert.AreEqual(ErrorMode.LocJobErr, ((GoodException) JobErrs.First()).Mode);
         }
@@ -122,7 +122,7 @@ namespace Apache.Ignite.Core.Tests.Compute
 
             Assert.AreEqual(1, res);
 
-            Assert.AreEqual(1, JobErrs.Count);
+            Assert.AreEqual(4, JobErrs.Count);
             Assert.IsNotNull(JobErrs.First() as BadException); // Local job exception is not marshalled.
         }
 
@@ -153,7 +153,7 @@ namespace Apache.Ignite.Core.Tests.Compute
 
             Assert.AreEqual(1, res);
 
-            Assert.AreEqual(1, JobErrs.Count);
+            Assert.AreEqual(4, JobErrs.Count);
 
             Assert.IsNotNull(JobErrs.ElementAt(0) as GoodException);
 
@@ -172,7 +172,7 @@ namespace Apache.Ignite.Core.Tests.Compute
 
             Assert.AreEqual(1, res);
 
-            Assert.AreEqual(1, JobErrs.Count);
+            Assert.AreEqual(4, JobErrs.Count);
 
             Assert.IsNotNull(JobErrs.ElementAt(0) as IgniteException);
         }
@@ -189,7 +189,7 @@ namespace Apache.Ignite.Core.Tests.Compute
 
             Assert.AreEqual(1, res);
 
-            Assert.AreEqual(1, JobErrs.Count);
+            Assert.AreEqual(4, JobErrs.Count);
 
             Assert.IsNotNull(JobErrs.ElementAt(0) as IgniteException);
         }
@@ -305,9 +305,19 @@ namespace Apache.Ignite.Core.Tests.Compute
         {
             JobErrs.Clear();
 
-            object res = Grid1.GetCompute().Execute(new Task());
+            Func<object, int> getRes = r => r is GoodTaskResult ? ((GoodTaskResult) r).Res : ((BadTaskResult) r).Res;
 
-            return res is GoodTaskResult ? ((GoodTaskResult)res).Res : ((BadTaskResult)res).Res;
+            var res1 = getRes(Grid1.GetCompute().Execute(new Task()));
+            var res2 = getRes(Grid1.GetCompute().Execute<object, object>(typeof(Task)));
+
+            var resAsync1 = getRes(Grid1.GetCompute().ExecuteAsync(new Task()).Result);
+            var resAsync2 = getRes(Grid1.GetCompute().ExecuteAsync<object, object>(typeof(Task)).Result);
+
+            Assert.AreEqual(res1, res2);
+            Assert.AreEqual(res2, resAsync1);
+            Assert.AreEqual(resAsync1, resAsync2);
+
+            return res1;
         }
 
         /// <summary>
@@ -318,20 +328,7 @@ namespace Apache.Ignite.Core.Tests.Compute
         {
             JobErrs.Clear();
 
-            Exception err = null;
-
-            try
-            {
-                Grid1.GetCompute().Execute(new Task());
-
-                Assert.Fail();
-            }
-            catch (Exception e)
-            {
-                err = e;
-            }
-
-            return err;
+            return Assert.Catch(() => Grid1.GetCompute().Execute(new Task()));
         }
 
         /// <summary>
@@ -391,11 +388,11 @@ namespace Apache.Ignite.Core.Tests.Compute
         /// <summary>
         /// Task.
         /// </summary>
-        public class Task : IComputeTask<object, object>
+        private class Task : IComputeTask<object, object>
         {
             /** Grid. */
             [InstanceResource]
-            private IIgnite _grid = null;
+            private readonly IIgnite _grid = null;
 
             /** Result. */
             private int _res;

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ResourceTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ResourceTaskTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ResourceTaskTest.cs
index 522341a..433b635 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ResourceTaskTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ResourceTaskTest.cs
@@ -54,6 +54,17 @@ namespace Apache.Ignite.Core.Tests.Compute
         }
 
         /// <summary>
+        /// Test Ignite injection into the task.
+        /// </summary>
+        [Test]
+        public void TestTaskInjectionBinarizable()
+        {
+            int res = Grid1.GetCompute().Execute(new InjectionTaskBinarizable(), 0);
+
+            Assert.AreEqual(GetServerCount(), res);
+        }
+
+        /// <summary>
         /// Test Ignite injection into the closure.
         /// </summary>
         [Test]
@@ -86,6 +97,12 @@ namespace Apache.Ignite.Core.Tests.Compute
             Assert.AreEqual(GetServerCount(), res);
         }
 
+        /** <inheritdoc /> */
+        protected override ICollection<Type> GetBinaryTypes()
+        {
+            return new[] {typeof(InjectionJobBinarizable)};
+        }
+
         /// <summary>
         /// Injection task.
         /// </summary>
@@ -113,6 +130,40 @@ namespace Apache.Ignite.Core.Tests.Compute
         }
 
         /// <summary>
+        /// Injection task.
+        /// </summary>
+        private class InjectionTaskBinarizable : Injectee, IComputeTask<object, int, int>
+        {
+            /** <inheritDoc /> */
+            public IDictionary<IComputeJob<int>, IClusterNode> Map(IList<IClusterNode> subgrid, object arg)
+            {
+                CheckInjection();
+
+                return subgrid.ToDictionary(x => (IComputeJob<int>) new InjectionJobBinarizable(), x => x);
+            }
+
+            /** <inheritDoc /> */
+            public ComputeJobResultPolicy OnResult(IComputeJobResult<int> res, IList<IComputeJobResult<int>> rcvd)
+            {
+                return ComputeJobResultPolicy.Wait;
+            }
+
+            /** <inheritDoc /> */
+            public int Reduce(IList<IComputeJobResult<int>> results)
+            {
+                return results.Sum(res => res.Data);
+            }
+        }
+
+        /// <summary>
+        /// Binarizable job.
+        /// </summary>
+        public class InjectionJobBinarizable : InjectionJob
+        {
+            // No-op.
+        }
+
+        /// <summary>
         /// Injection job.
         /// </summary>
         [Serializable]

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/SerializableClosureTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/SerializableClosureTaskTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/SerializableClosureTaskTest.cs
index ded56ed..8db4876 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/SerializableClosureTaskTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/SerializableClosureTaskTest.cs
@@ -66,6 +66,11 @@ namespace Apache.Ignite.Core.Tests.Compute
         {
             Assert.IsTrue(err != null);
 
+            var aggregate = err as AggregateException;
+
+            if (aggregate != null)
+                err = aggregate.InnerException;
+
             SerializableException err0 = err as SerializableException;
 
             Assert.IsTrue(err0 != null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/TaskAdapterTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/TaskAdapterTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/TaskAdapterTest.cs
index 7789ac4..32a28a7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/TaskAdapterTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/TaskAdapterTest.cs
@@ -22,7 +22,6 @@ namespace Apache.Ignite.Core.Tests.Compute
     using System;
     using System.Collections.Generic;
     using System.Linq;
-    using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Compute;
     using Apache.Ignite.Core.Resource;
     using NUnit.Framework;
@@ -102,9 +101,9 @@ namespace Apache.Ignite.Core.Tests.Compute
         }
 
         /** <inheritDoc /> */
-        override protected void GetBinaryTypeConfigurations(ICollection<BinaryTypeConfiguration> portTypeCfgs)
+        protected override ICollection<Type> GetBinaryTypes()
         {
-            portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableJob)));
+            return new[] { typeof(BinarizableJob) };
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/TaskResultTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/TaskResultTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/TaskResultTest.cs
index 26286de..289b68b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/TaskResultTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/TaskResultTest.cs
@@ -19,7 +19,6 @@ namespace Apache.Ignite.Core.Tests.Compute
 {
     using System;
     using System.Collections.Generic;
-    using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Compute;
     using Apache.Ignite.Core.Resource;
@@ -156,12 +155,15 @@ namespace Apache.Ignite.Core.Tests.Compute
         }
 
         /** <inheritDoc /> */
-        override protected void GetBinaryTypeConfigurations(ICollection<BinaryTypeConfiguration> portTypeCfgs)
+        protected override ICollection<Type> GetBinaryTypes()
         {
-            portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableResult)));
-            portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(TestBinarizableJob)));
-            portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableOutFunc)));
-            portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableFunc)));
+            return new[]
+            {
+                typeof(BinarizableResult),
+                typeof(TestBinarizableJob),
+                typeof(BinarizableOutFunc),
+                typeof(BinarizableFunc)
+            };
         }
 
         [Test]

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-standalone.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-standalone.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-standalone.xml
index af5f499..3990e3b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-standalone.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Compute/compute-standalone.xml
@@ -68,6 +68,7 @@
                                 <value>Apache.Ignite.Core.Tests.Compute.BinarizableClosureTaskTest+BinarizableFunc</value>
                                 <value>Apache.Ignite.Core.Tests.Compute.BinarizableClosureTaskTest+BinarizableResult</value>
                                 <value>Apache.Ignite.Core.Tests.Compute.BinarizableClosureTaskTest+BinarizableException</value>
+                                <value>Apache.Ignite.Core.Tests.Compute.ResourceTaskTest+InjectionJobBinarizable</value>
                             </list>
                         </property>
                         <property name="typesConfiguration">

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
index c2ccd1a..cc21490 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+#pragma warning disable 618
 namespace Apache.Ignite.Core.Tests
 {
     using System;
@@ -25,6 +26,7 @@ namespace Apache.Ignite.Core.Tests
     using System.Threading.Tasks;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cache.Query;
+    using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Events;
     using Apache.Ignite.Core.Impl;
@@ -141,7 +143,7 @@ namespace Apache.Ignite.Core.Tests
             CheckSend(2);
 
             // Unsubscribe from all events
-            events.StopLocalListen(listener);
+            events.StopLocalListen(listener, Enumerable.Empty<int>());
 
             CheckNoEvent();
 
@@ -299,10 +301,19 @@ namespace Apache.Ignite.Core.Tests
         }
 
         /// <summary>
+        /// Tests the record local.
+        /// </summary>
+        [Test]
+        public void TestRecordLocal()
+        {
+            Assert.Throws<NotImplementedException>(() => _grid1.GetEvents().RecordLocal(new MyEvent()));
+        }
+
+        /// <summary>
         /// Tests the WaitForLocal.
         /// </summary>
         [Test]
-        public void TestWaitForLocal([Values(true, false)] bool async)
+        public void TestWaitForLocal()
         {
             var events = _grid1.GetEvents();
 
@@ -312,50 +323,80 @@ namespace Apache.Ignite.Core.Tests
 
             events.EnableLocal(eventType);
 
-            Func<IEventFilter<IEvent>, int[], Task<IEvent>> getWaitTask;
+            var taskFuncs = GetWaitTasks(events).Select(
+                func => (Func<IEventFilter<IEvent>, int[], Task<IEvent>>) (
+                    (filter, types) =>
+                    {
+                        var task = func(filter, types);
 
-            if (async)
-                getWaitTask = (filter, types) =>
-                {
-                    var task = events.WaitForLocalAsync(filter, types);
-                    GenerateTaskEvent();
-                    return task;
-                };
-            else
-                getWaitTask = (filter, types) =>
+                        Thread.Sleep(100); // allow task to start and begin waiting for events
+
+                        GenerateTaskEvent();
+
+                        return task;
+                    })).ToArray();
+
+            for (int i = 0; i < taskFuncs.Length; i++)
+            {
+                var getWaitTask = taskFuncs[i];
+
+                // No params
+                var waitTask = getWaitTask(null, new int[0]);
+
+                waitTask.Wait(timeout);
+
+                // Event types
+                waitTask = getWaitTask(null, new[] {EventType.TaskReduced});
+
+                Assert.IsTrue(waitTask.Wait(timeout));
+                Assert.IsInstanceOf(typeof(TaskEvent), waitTask.Result);
+                Assert.AreEqual(EventType.TaskReduced, waitTask.Result.Type);
+
+                if (i > 3)
                 {
-                    var task = Task.Factory.StartNew(() => events.WaitForLocal(filter, types));
-                    Thread.Sleep(500); // allow task to start and begin waiting for events
-                    GenerateTaskEvent();
-                    return task;
-                };
+                    // Filter
+                    waitTask = getWaitTask(new EventFilter<IEvent>(e => e.Type == EventType.TaskReduced), new int[0]);
 
-            // No params
-            var waitTask = getWaitTask(null, new int[0]);
+                    Assert.IsTrue(waitTask.Wait(timeout));
+                    Assert.IsInstanceOf(typeof(TaskEvent), waitTask.Result);
+                    Assert.AreEqual(EventType.TaskReduced, waitTask.Result.Type);
 
-            waitTask.Wait(timeout);
+                    // Filter & types
+                    waitTask = getWaitTask(new EventFilter<IEvent>(e => e.Type == EventType.TaskReduced),
+                        new[] {EventType.TaskReduced});
 
-            // Event types
-            waitTask = getWaitTask(null, new[] {EventType.TaskReduced});
+                    Assert.IsTrue(waitTask.Wait(timeout));
+                    Assert.IsInstanceOf(typeof(TaskEvent), waitTask.Result);
+                    Assert.AreEqual(EventType.TaskReduced, waitTask.Result.Type);
+                }
+            }
+        }
 
-            Assert.IsTrue(waitTask.Wait(timeout));
-            Assert.IsInstanceOf(typeof(TaskEvent), waitTask.Result);
-            Assert.AreEqual(EventType.TaskReduced, waitTask.Result.Type);
+        /// <summary>
+        /// Gets the wait tasks for different overloads of WaitForLocal.
+        /// </summary>
+        private static IEnumerable<Func<IEventFilter<IEvent>, int[], Task<IEvent>>> GetWaitTasks(IEvents events)
+        {
+            yield return (filter, types) => Task.Factory.StartNew(() => events.WaitForLocal(types));
+            yield return (filter, types) => Task.Factory.StartNew(() => events.WaitForLocal(types.ToList()));
 
-            // Filter
-            waitTask = getWaitTask(new EventFilter<IEvent>(e => e.Type == EventType.TaskReduced), new int[0]);
+            yield return (filter, types) => events.WaitForLocalAsync(types);
+            yield return (filter, types) => events.WaitForLocalAsync(types.ToList());
 
-            Assert.IsTrue(waitTask.Wait(timeout));
-            Assert.IsInstanceOf(typeof(TaskEvent), waitTask.Result);
-            Assert.AreEqual(EventType.TaskReduced, waitTask.Result.Type);
+            yield return (filter, types) => Task.Factory.StartNew(() => events.WaitForLocal(filter, types));
+            yield return (filter, types) => Task.Factory.StartNew(() => events.WaitForLocal(filter, types.ToList()));
 
-            // Filter & types
-            waitTask = getWaitTask(new EventFilter<IEvent>(e => e.Type == EventType.TaskReduced),
-                new[] {EventType.TaskReduced});
+            yield return (filter, types) => events.WaitForLocalAsync(filter, types);
+            yield return (filter, types) => events.WaitForLocalAsync(filter, types.ToList());
+        }
 
-            Assert.IsTrue(waitTask.Wait(timeout));
-            Assert.IsInstanceOf(typeof(TaskEvent), waitTask.Result);
-            Assert.AreEqual(EventType.TaskReduced, waitTask.Result.Type);
+        /// <summary>
+        /// Tests the wait for local overloads.
+        /// </summary>
+        [Test]
+        public void TestWaitForLocalOverloads()
+        {
+            
         }
 
         /*
@@ -492,6 +533,7 @@ namespace Apache.Ignite.Core.Tests
                 Assert.AreEqual(expectedGuid, cacheEvent.SubjectId);
                 Assert.AreEqual("cloClsName", cacheEvent.ClosureClassName);
                 Assert.AreEqual("taskName", cacheEvent.TaskName);
+                Assert.IsTrue(cacheEvent.ToShortString().StartsWith("SWAP_SPACE_CLEARED: IsNear="));
 
                 var qryExecEvent = EventReader.Read<CacheQueryExecutedEvent>(reader);
                 CheckEventBase(qryExecEvent);
@@ -501,6 +543,9 @@ namespace Apache.Ignite.Core.Tests
                 Assert.AreEqual("clause", qryExecEvent.Clause);
                 Assert.AreEqual(expectedGuid, qryExecEvent.SubjectId);
                 Assert.AreEqual("taskName", qryExecEvent.TaskName);
+                Assert.AreEqual(
+                    "SWAP_SPACE_CLEARED: QueryType=qryType, CacheName=cacheName, ClassName=clsName, Clause=clause, " +
+                    "SubjectId=00000000-0000-0001-0000-000000000002, TaskName=taskName", qryExecEvent.ToShortString());
 
                 var qryReadEvent = EventReader.Read<CacheQueryReadEvent>(reader);
                 CheckEventBase(qryReadEvent);
@@ -514,6 +559,10 @@ namespace Apache.Ignite.Core.Tests
                 Assert.AreEqual(2, qryReadEvent.Value);
                 Assert.AreEqual(3, qryReadEvent.OldValue);
                 Assert.AreEqual(4, qryReadEvent.Row);
+                Assert.AreEqual(
+                    "SWAP_SPACE_CLEARED: QueryType=qryType, CacheName=cacheName, ClassName=clsName, Clause=clause, " +
+                    "SubjectId=00000000-0000-0001-0000-000000000002, TaskName=taskName, Key=1, Value=2, " +
+                    "OldValue=3, Row=4", qryReadEvent.ToShortString());
 
                 var cacheRebalancingEvent = EventReader.Read<CacheRebalancingEvent>(reader);
                 CheckEventBase(cacheRebalancingEvent);
@@ -522,15 +571,19 @@ namespace Apache.Ignite.Core.Tests
                 Assert.AreEqual(locNode, cacheRebalancingEvent.DiscoveryNode);
                 Assert.AreEqual(2, cacheRebalancingEvent.DiscoveryEventType);
                 Assert.AreEqual(3, cacheRebalancingEvent.DiscoveryTimestamp);
-                
+                Assert.IsTrue(cacheRebalancingEvent.ToShortString().StartsWith(
+                    "SWAP_SPACE_CLEARED: CacheName=cacheName, Partition=1, DiscoveryNode=GridNode"));
+
                 var checkpointEvent = EventReader.Read<CheckpointEvent>(reader);
                 CheckEventBase(checkpointEvent);
                 Assert.AreEqual("cpKey", checkpointEvent.Key);
-                
+                Assert.AreEqual("SWAP_SPACE_CLEARED: Key=cpKey", checkpointEvent.ToShortString());
+
                 var discoEvent = EventReader.Read<DiscoveryEvent>(reader);
                 CheckEventBase(discoEvent);
                 Assert.AreEqual(grid.TopologyVersion, discoEvent.TopologyVersion);
                 Assert.AreEqual(grid.GetNodes(), discoEvent.TopologyNodes);
+                Assert.IsTrue(discoEvent.ToShortString().StartsWith("SWAP_SPACE_CLEARED: EventNode=GridNode"));
 
                 var jobEvent = EventReader.Read<JobEvent>(reader);
                 CheckEventBase(jobEvent);
@@ -540,10 +593,12 @@ namespace Apache.Ignite.Core.Tests
                 Assert.AreEqual(locNode, jobEvent.TaskNode);
                 Assert.AreEqual(expectedGridGuid, jobEvent.TaskSessionId);
                 Assert.AreEqual(expectedGuid, jobEvent.TaskSubjectId);
+                Assert.IsTrue(jobEvent.ToShortString().StartsWith("SWAP_SPACE_CLEARED: TaskName=taskName"));
 
                 var spaceEvent = EventReader.Read<SwapSpaceEvent>(reader);
                 CheckEventBase(spaceEvent);
                 Assert.AreEqual("space", spaceEvent.Space);
+                Assert.IsTrue(spaceEvent.ToShortString().StartsWith("SWAP_SPACE_CLEARED: Space=space"));
 
                 var taskEvent = EventReader.Read<TaskEvent>(reader);
                 CheckEventBase(taskEvent);
@@ -552,6 +607,7 @@ namespace Apache.Ignite.Core.Tests
                 Assert.AreEqual("taskClsName", taskEvent.TaskClassName);
                 Assert.AreEqual("taskName", taskEvent.TaskName);
                 Assert.AreEqual(expectedGridGuid, taskEvent.TaskSessionId);
+                Assert.IsTrue(taskEvent.ToShortString().StartsWith("SWAP_SPACE_CLEARED: TaskName=taskName"));
             }
         }
 
@@ -570,6 +626,11 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreNotEqual(Guid.Empty, evt.Id.GlobalId);
             Assert.IsTrue(Math.Abs((evt.Timestamp - DateTime.UtcNow).TotalSeconds) < 20, 
                 "Invalid event timestamp: '{0}', current time: '{1}'", evt.Timestamp, DateTime.Now);
+
+            Assert.Greater(evt.LocalOrder, 0);
+
+            Assert.IsTrue(evt.ToString().Contains("[Name=SWAP_SPACE_CLEARED"));
+            Assert.IsTrue(evt.ToShortString().StartsWith("SWAP_SPACE_CLEARED"));
         }
 
         /// <summary>
@@ -855,6 +916,7 @@ namespace Apache.Ignite.Core.Tests
         }
 
         /** <inheritdoc /> */
+        // ReSharper disable once UnusedMember.Global
         public bool Invoke(T evt)
         {
             throw new Exception("Invalid method");
@@ -954,4 +1016,58 @@ namespace Apache.Ignite.Core.Tests
             return EventObjectType.ToString();
         }
     }
+
+    /// <summary>
+    /// Custom event.
+    /// </summary>
+    public class MyEvent : IEvent
+    {
+        /** <inheritdoc /> */
+        public IgniteGuid Id
+        {
+            get { throw new NotImplementedException(); }
+        }
+
+        /** <inheritdoc /> */
+        public long LocalOrder
+        {
+            get { throw new NotImplementedException(); }
+        }
+
+        /** <inheritdoc /> */
+        public IClusterNode Node
+        {
+            get { throw new NotImplementedException(); }
+        }
+
+        /** <inheritdoc /> */
+        public string Message
+        {
+            get { throw new NotImplementedException(); }
+        }
+
+        /** <inheritdoc /> */
+        public int Type
+        {
+            get { throw new NotImplementedException(); }
+        }
+
+        /** <inheritdoc /> */
+        public string Name
+        {
+            get { throw new NotImplementedException(); }
+        }
+
+        /** <inheritdoc /> */
+        public DateTime Timestamp
+        {
+            get { throw new NotImplementedException(); }
+        }
+
+        /** <inheritdoc /> */
+        public string ToShortString()
+        {
+            throw new NotImplementedException();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs
index 02a5d0b..f6730d7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs
@@ -27,7 +27,7 @@ namespace Apache.Ignite.Core.Tests.Examples
     /// <summary>
     /// Tests all examples in various modes.
     /// </summary>
-    [Category(TestUtils.CategoryIntensive)]
+    [Category(TestUtils.CategoryExamples)]
     public class ExamplesTest
     {
         /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs
index a324191..e766f5a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs
@@ -122,6 +122,52 @@ namespace Apache.Ignite.Core.Tests
         }
 
         /// <summary>
+        /// Tests that all exceptions have mandatory constructors and are serializable.
+        /// </summary>
+        [Test]
+        public void TestAllExceptionsConstructors()
+        {
+            var types = typeof(IIgnite).Assembly.GetTypes().Where(x => x.IsSubclassOf(typeof(Exception)));
+
+            foreach (var type in types)
+            {
+                Assert.IsTrue(type.IsSerializable, "Exception is not serializable: " + type);
+
+                // Default ctor.
+                var defCtor = type.GetConstructor(new Type[0]);
+                Assert.IsNotNull(defCtor);
+
+                var ex = (Exception) defCtor.Invoke(new object[0]);
+                Assert.AreEqual(string.Format("Exception of type '{0}' was thrown.", type.FullName), ex.Message);
+
+                // Message ctor.
+                var msgCtor = type.GetConstructor(new[] {typeof(string)});
+                Assert.IsNotNull(msgCtor);
+
+                ex = (Exception) msgCtor.Invoke(new object[] {"myMessage"});
+                Assert.AreEqual("myMessage", ex.Message);
+
+                // Serialization.
+                var stream = new MemoryStream();
+                var formatter = new BinaryFormatter();
+
+                formatter.Serialize(stream, ex);
+                stream.Seek(0, SeekOrigin.Begin);
+
+                ex = (Exception) formatter.Deserialize(stream);
+                Assert.AreEqual("myMessage", ex.Message);
+
+                // Message+cause ctor.
+                var msgCauseCtor = type.GetConstructor(new[] { typeof(string), typeof(Exception) });
+                Assert.IsNotNull(msgCauseCtor);
+
+                ex = (Exception) msgCauseCtor.Invoke(new object[] {"myMessage", new Exception("innerEx")});
+                Assert.AreEqual("myMessage", ex.Message);
+                Assert.AreEqual("innerEx", ex.InnerException.Message);
+            }
+        }
+
+        /// <summary>
         /// Tests CachePartialUpdateException serialization.
         /// </summary>
         private static void TestPartialUpdateExceptionSerialization(Exception ex)

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
index d3851db..88a2b52 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
@@ -39,6 +39,9 @@ namespace Apache.Ignite.Core.Tests
         /** Indicates long running and/or memory/cpu intensive test. */
         public const string CategoryIntensive = "LONG_TEST";
 
+        /** Indicates examples tests. */
+        public const string CategoryExamples = "EXAMPLES_TEST";
+
         /** */
         public const int DfltBusywaitSleepInterval = 200;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 6fa1378..8e8f8ca 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -310,7 +310,6 @@
     <Compile Include="Impl\Common\IgniteConfigurationXmlSerializer.cs" />
     <Compile Include="Impl\Common\IgniteHome.cs" />
     <Compile Include="Impl\Common\LoadedAssembliesResolver.cs" />
-    <Compile Include="Impl\Common\ResizeableArray.cs" />
     <Compile Include="Impl\Common\TypeCaster.cs" />
     <Compile Include="Impl\Common\TypeStringConverter.cs" />
     <Compile Include="Impl\Compute\Closure\ComputeAbstractClosureTask.cs" />
@@ -351,7 +350,6 @@
     <Compile Include="Impl\Handle\HandleRegistry.cs" />
     <Compile Include="Impl\Handle\IHandle.cs" />
     <Compile Include="Impl\IInteropCallback.cs" />
-    <Compile Include="Impl\InteropExceptionHolder.cs" />
     <Compile Include="Impl\LifecycleBeanHolder.cs" />
     <Compile Include="Impl\Memory\InteropExternalMemory.cs" />
     <Compile Include="Impl\Memory\InteropMemoryUtils.cs" />
@@ -386,7 +384,6 @@
     <Compile Include="Impl\Binary\BinarizableSerializer.cs" />
     <Compile Include="Impl\Binary\Marshaller.cs" />
     <Compile Include="Impl\Binary\BinaryMode.cs" />
-    <Compile Include="Impl\Binary\BinaryObjectHandle.cs" />
     <Compile Include="Impl\Binary\BinaryObjectHeader.cs" />
     <Compile Include="Impl\Binary\BinaryObjectSchema.cs" />
     <Compile Include="Impl\Binary\BinaryObjectSchemaField.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/AffinityKey.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/AffinityKey.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/AffinityKey.cs
index 1d27d65..1b7fcb0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/AffinityKey.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Affinity/AffinityKey.cs
@@ -158,5 +158,16 @@ namespace Apache.Ignite.Core.Cache.Affinity
         {
             return !left.Equals(right);
         }
+
+        /// <summary>
+        /// Returns a <see cref="string" /> that represents this instance.
+        /// </summary>
+        /// <returns>
+        /// A <see cref="string" /> that represents this instance.
+        /// </returns>
+        public override string ToString()
+        {
+            return string.Format("AffinityKey [Key={0}, Affinity={1}]", _key, _affinity);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs
index 907af14..484fceb 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/CachePartialUpdateException.cs
@@ -71,7 +71,8 @@ namespace Apache.Ignite.Core.Cache
         /// </summary>
         /// <param name="msg">Exception message.</param>
         /// <param name="failedKeysException">Exception occurred during failed keys read/write.</param>
-        public CachePartialUpdateException(string msg, Exception failedKeysException) : this(msg, null, failedKeysException)
+        public CachePartialUpdateException(string msg, Exception failedKeysException) 
+            : this(msg, null, failedKeysException)
         {
             // No-op.
         }
@@ -92,7 +93,8 @@ namespace Apache.Ignite.Core.Cache
         /// <param name="msg">Exception message.</param>
         /// <param name="failedKeys">Failed keys.</param>
         /// <param name="failedKeysException">Exception occurred during failed keys read/write.</param>
-        private CachePartialUpdateException(string msg, IList<object> failedKeys, Exception failedKeysException) : base(msg)
+        private CachePartialUpdateException(string msg, IList<object> failedKeys, Exception failedKeysException) 
+            : base(msg, failedKeysException)
         {
             _failedKeys = failedKeys;
             _failedKeysException = failedKeysException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Store/CacheParallelLoadStoreAdapter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Store/CacheParallelLoadStoreAdapter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Store/CacheParallelLoadStoreAdapter.cs
index 147f1bd..c506838 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Store/CacheParallelLoadStoreAdapter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Store/CacheParallelLoadStoreAdapter.cs
@@ -110,6 +110,7 @@ namespace Apache.Ignite.Core.Cache.Store
         /// The value for the entry that is to be stored in the cache
         /// or <c>null</c> if the object can't be loaded
         /// </returns>
+        [ExcludeFromCodeCoverage]
         public virtual object Load(object key)
         {
             return null;
@@ -124,6 +125,7 @@ namespace Apache.Ignite.Core.Cache.Store
         /// <returns>
         /// A map of key, values to be stored in the cache.
         /// </returns>
+        [ExcludeFromCodeCoverage]
         public virtual IDictionary LoadAll(ICollection keys)
         {
             return null;
@@ -136,6 +138,7 @@ namespace Apache.Ignite.Core.Cache.Store
         /// </summary>
         /// <param name="key">Key to write.</param>
         /// <param name="val">Value to write.</param>
+        [ExcludeFromCodeCoverage]
         public virtual void Write(object key, object val)
         {
             // No-op.
@@ -154,6 +157,7 @@ namespace Apache.Ignite.Core.Cache.Store
         /// <param name="entries">a mutable collection to write. Upon invocation,  it contains the entries
         /// to write for write-through. Upon return the collection must only contain entries
         /// that were not successfully written. (see partial success above).</param>
+        [ExcludeFromCodeCoverage]
         public virtual void WriteAll(IDictionary entries)
         {
             // No-op.
@@ -167,6 +171,7 @@ namespace Apache.Ignite.Core.Cache.Store
         /// This method is invoked even if no mapping for the key exists.
         /// </summary>
         /// <param name="key">The key that is used for the delete operation.</param>
+        [ExcludeFromCodeCoverage]
         public virtual void Delete(object key)
         {
             // No-op.
@@ -189,6 +194,7 @@ namespace Apache.Ignite.Core.Cache.Store
         /// <param name="keys">a mutable collection of keys for entries to delete. Upon invocation,
         /// it contains the keys to delete for write-through. Upon return the collection must only contain
         /// the keys that were not successfully deleted.</param>
+        [ExcludeFromCodeCoverage]
         public virtual void DeleteAll(ICollection keys)
         {
             // No-op.
@@ -199,6 +205,7 @@ namespace Apache.Ignite.Core.Cache.Store
         /// <c>commit</c> parameter.
         /// </summary>
         /// <param name="commit"><c>True</c> if transaction should commit, <c>false</c> for rollback.</param>
+        [ExcludeFromCodeCoverage]
         public virtual void SessionEnd(bool commit)
         {
             // No-op.

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
index 13d3133..39a2f8b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
@@ -21,6 +21,7 @@ namespace Apache.Ignite.Core.Impl.Binary
     using System.Collections;
     using System.Collections.Generic;
     using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
     using System.IO;
     using System.Runtime.CompilerServices;
     using System.Text;
@@ -125,6 +126,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /** <inheritdoc /> */
+        [ExcludeFromCodeCoverage]
         public int EnumValue
         {
             get

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
index 646d563..1626a2d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
@@ -1027,7 +1027,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /// <summary>
-        /// Mutation ocntext.
+        /// Mutation context.
         /// </summary>
         private class Context
         {

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHandle.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHandle.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHandle.cs
deleted file mode 100644
index 35735fe..0000000
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHandle.cs
+++ /dev/null
@@ -1,59 +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.
- */
-
-namespace Apache.Ignite.Core.Impl.Binary
-{
-    /// <summary>
-    /// Object handle. Wraps a single value.
-    /// </summary>
-    internal class BinaryObjectHandle
-    {
-        /** Value. */
-        private readonly object _val;
-
-        /// <summary>
-        /// Initializes a new instance of the <see cref="BinaryObjectHandle"/> class.
-        /// </summary>
-        /// <param name="val">The value.</param>
-        public BinaryObjectHandle(object val)
-        {
-            _val = val;
-        }
-
-        /// <summary>
-        /// Gets the value.
-        /// </summary>
-        public object Value
-        {
-            get { return _val; }
-        }
-
-        /** <inheritdoc /> */
-        public override bool Equals(object obj)
-        {
-            var that = obj as BinaryObjectHandle;
-
-            return that != null && _val == that._val;
-        }
-
-        /** <inheritdoc /> */
-        public override int GetHashCode()
-        {
-            return _val != null ? _val.GetHashCode() : 0;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
index 2624d52..bb5c207 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
@@ -19,6 +19,7 @@ namespace Apache.Ignite.Core.Impl.Binary
 {
     using System;
     using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
     using System.IO;
     using System.Runtime.InteropServices;
     using Apache.Ignite.Core.Impl.Binary.IO;
@@ -100,6 +101,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// Initializes a new instance of the <see cref="BinaryObjectHeader"/> struct from specified stream.
         /// </summary>
         /// <param name="stream">The stream.</param>
+        [ExcludeFromCodeCoverage]   // big-endian only
         private BinaryObjectHeader(IBinaryStream stream)
         {
             Header = stream.ReadByte();
@@ -116,6 +118,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// Writes this instance to the specified stream.
         /// </summary>
         /// <param name="stream">The stream.</param>
+        [ExcludeFromCodeCoverage]   // big-endian only
         private void Write(IBinaryStream stream)
         {
             stream.WriteByte(Header);
@@ -292,7 +295,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         public override bool Equals(object obj)
         {
             if (ReferenceEquals(null, obj)) return false;
-            
+
             return obj is BinaryObjectHeader && Equals((BinaryObjectHeader) obj);
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySurrogateTypeDescriptor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySurrogateTypeDescriptor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySurrogateTypeDescriptor.cs
index 16e3032..b572e7c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySurrogateTypeDescriptor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySurrogateTypeDescriptor.cs
@@ -58,7 +58,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /// <summary>
-        /// Constrcutor.
+        /// Constructor.
         /// </summary>
         /// <param name="cfg">Configuration.</param>
         /// <param name="name">Type name.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
index 585ccd3..77a22dd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
@@ -1442,26 +1442,6 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /// <summary>
-        /// Check whether the given object is binarizeble, i.e. it can be serialized with binary marshaller.
-        /// </summary>
-        /// <param name="obj">Object.</param>
-        /// <returns>True if binarizable.</returns>
-        internal bool IsBinarizable(object obj)
-        {
-            if (obj != null)
-            {
-                Type type = obj.GetType();
-
-                // We assume object as binarizable only in case it has descriptor.
-                // Collections, Enums and non-primitive arrays do not have descriptors
-                // and this is fine here because we cannot know whether their members are binarizable.
-                return _marsh.GetDescriptor(type) != null || BinarySystemHandlers.GetWriteHandler(type) != null;
-            }
-
-            return true;
-        }
-
-        /// <summary>
         /// Write field ID.
         /// </summary>
         /// <param name="fieldName">Field name.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamAdapter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamAdapter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamAdapter.cs
index dcbff81..b062689 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamAdapter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamAdapter.cs
@@ -18,6 +18,7 @@
 namespace Apache.Ignite.Core.Impl.Binary.IO
 {
     using System;
+    using System.Diagnostics.CodeAnalysis;
     using System.IO;
 
     /// <summary>
@@ -78,12 +79,14 @@ namespace Apache.Ignite.Core.Impl.Binary.IO
         }
 
         /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
         public override long Seek(long offset, SeekOrigin origin)
         {
             throw new NotSupportedException("Stream is not seekable.");
         }
 
         /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
         public override long Position
         {
             get
@@ -97,6 +100,7 @@ namespace Apache.Ignite.Core.Impl.Binary.IO
         }
 
         /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
         public override long Length
         {
             get 
@@ -106,6 +110,7 @@ namespace Apache.Ignite.Core.Impl.Binary.IO
         }
 
         /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
         public override void SetLength(long value)
         {
             throw new NotSupportedException("Stream is not seekable.");

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamBase.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamBase.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamBase.cs
index a5e140d..6286602 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamBase.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamBase.cs
@@ -25,7 +25,7 @@ namespace Apache.Ignite.Core.Impl.Binary.IO
     /// <summary>
     /// Base class for managed and unmanaged data streams.
     /// </summary>
-    internal unsafe abstract class BinaryStreamBase : IBinaryStream
+    internal abstract unsafe class BinaryStreamBase : IBinaryStream
     {
         /** Byte: zero. */
         private const byte ByteZero = 0;
@@ -1070,13 +1070,10 @@ namespace Apache.Ignite.Core.Impl.Binary.IO
         /// <returns>
         ///   <c>True</c> if they are same.
         /// </returns>
-        public virtual bool IsSameArray(byte[] arr)
-        {
-            return false;
-        }
-
+        public abstract bool IsSameArray(byte[] arr);
+        
         /// <summary>
-        /// Seek to the given positoin.
+        /// Seek to the given position.
         /// </summary>
         /// <param name="offset">Offset.</param>
         /// <param name="origin">Seek origin.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStream.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStream.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStream.cs
index d530713..cd509c6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStream.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStream.cs
@@ -312,7 +312,7 @@ namespace Apache.Ignite.Core.Impl.Binary.IO
         bool IsSameArray(byte[] arr);
 
         /// <summary>
-        /// Seek to the given positoin.
+        /// Seek to the given position.
         /// </summary>
         /// <param name="offset">Offset.</param>
         /// <param name="origin">Seek origin.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/JavaTypes.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/JavaTypes.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/JavaTypes.cs
index a8d94f2..109d55f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/JavaTypes.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/JavaTypes.cs
@@ -62,9 +62,6 @@ namespace Apache.Ignite.Core.Impl.Binary
         private static readonly Dictionary<string, Type> JavaToNet =
             NetToJava.GroupBy(x => x.Value).ToDictionary(g => g.Key, g => g.First().Key);
 
-        /** */
-        private static readonly string MappedTypes = string.Join(", ", NetToJava.Keys.Select(x => x.Name));
-
         /// <summary>
         /// Gets the corresponding Java type name.
         /// </summary>
@@ -110,13 +107,5 @@ namespace Apache.Ignite.Core.Impl.Binary
 
             return JavaToNet.TryGetValue(javaTypeName, out res) ? res : null;
         }
-
-        /// <summary>
-        /// Gets the supported types as a comma-separated string.
-        /// </summary>
-        public static string SupportedTypesString
-        {
-            get { return MappedTypes; }
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Event/JavaCacheEntryEventFilter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Event/JavaCacheEntryEventFilter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Event/JavaCacheEntryEventFilter.cs
index b5c2ece..a56e12a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Event/JavaCacheEntryEventFilter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Event/JavaCacheEntryEventFilter.cs
@@ -19,6 +19,7 @@ namespace Apache.Ignite.Core.Impl.Cache.Event
 {
     using System;
     using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
     using Apache.Ignite.Core.Cache.Event;
     using Apache.Ignite.Core.Impl.Common;
 
@@ -30,6 +31,7 @@ namespace Apache.Ignite.Core.Impl.Cache.Event
     internal class JavaCacheEntryEventFilter<TK, TV> : PlatformJavaObjectFactoryProxy, ICacheEntryEventFilter<TK, TV>
     {
         /** <inheritdoc /> */
+        [ExcludeFromCodeCoverage]
         public bool Evaluate(ICacheEntryEvent<TK, TV> evt)
         {
             throw new InvalidOperationException(GetType() + " cannot be invoked directly.");

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Collections/MultiValueDictionary.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Collections/MultiValueDictionary.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Collections/MultiValueDictionary.cs
index 2adb021..a4e9c93 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Collections/MultiValueDictionary.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Collections/MultiValueDictionary.cs
@@ -52,32 +52,6 @@ namespace Apache.Ignite.Core.Impl.Collections
         }
 
         /// <summary>
-        /// Tries the get a value. In case of multiple values for a key, returns the last one.
-        /// </summary>
-        /// <param name="key">The key.</param>
-        /// <param name="val">The value.</param>
-        /// <returns>True if value has been found for specified key; otherwise false.</returns>
-        public bool TryGetValue(TKey key, out TValue val)
-        {
-            object val0;
-            
-            if (!_dict.TryGetValue(key, out val0))
-            {
-                val = default(TValue);
-                return false;
-            }
-
-            var list = val0 as List<TValue>;
-
-            if (list != null)
-                val = list[list.Count - 1];
-            else
-                val = (TValue) val0;
-
-            return true;
-        }
-
-        /// <summary>
         /// Removes the specified value for the specified key.
         /// </summary>
         /// <param name="key">The key.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Collections/ReadOnlyDictionary.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Collections/ReadOnlyDictionary.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Collections/ReadOnlyDictionary.cs
index 60ec9d0..a51a149 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Collections/ReadOnlyDictionary.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Collections/ReadOnlyDictionary.cs
@@ -110,7 +110,7 @@ namespace Apache.Ignite.Core.Impl.Collections
         /** <inheritdoc /> */
         public bool Remove(TKey key)
         {
-            return _dict.Remove(key);
+            throw GetReadonlyException();
         }
 
         /** <inheritdoc /> */

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/ResizeableArray.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/ResizeableArray.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/ResizeableArray.cs
deleted file mode 100644
index 82a8eee..0000000
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/ResizeableArray.cs
+++ /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.
- */
-
-namespace Apache.Ignite.Core.Impl.Common
-{
-    using System.Collections.Generic;
-
-    /// <summary>
-    /// Simple append-only <see cref="List{T}"/> alternative which exposes internal array.
-    /// </summary>
-    internal class ResizeableArray<T>
-    {
-        /** Array. */
-        private T[] _arr;
-
-        /// <summary>
-        /// Constructor.
-        /// </summary>
-        /// <param name="capacity">Capacity.</param>
-        public ResizeableArray(int capacity)
-        {
-            _arr = new T[capacity];
-        }
-
-        /// <summary>
-        /// Array.
-        /// </summary>
-        public T[] Array
-        {
-            get { return _arr; }
-        }
-
-        /// <summary>
-        /// Count.
-        /// </summary>
-        public int Count { get; private set; }
-
-        /// <summary>
-        /// Add element.
-        /// </summary>
-        /// <param name="element">Element.</param>
-        public void Add(T element)
-        {
-            if (Count == _arr.Length)
-                System.Array.Resize(ref _arr, _arr.Length*2);
-
-            _arr[Count++] = element;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/EventTypeConverter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/EventTypeConverter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/EventTypeConverter.cs
index 6b8f935..f0d1564 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/EventTypeConverter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/EventTypeConverter.cs
@@ -20,6 +20,7 @@ namespace Apache.Ignite.Core.Impl.Events
     using System;
     using System.Collections.Generic;
     using System.ComponentModel;
+    using System.Diagnostics.CodeAnalysis;
     using System.Globalization;
     using System.Linq;
     using Apache.Ignite.Core.Events;
@@ -57,6 +58,7 @@ namespace Apache.Ignite.Core.Impl.Events
         /// <returns>
         /// true if this converter can perform the conversion; otherwise, false.
         /// </returns>
+        [ExcludeFromCodeCoverage]  // not called
         public override bool CanConvertFrom(ITypeDescriptorContext context, Type sourceType)
         {
             return sourceType == typeof(string);
@@ -72,6 +74,7 @@ namespace Apache.Ignite.Core.Impl.Events
         /// <returns>
         /// true if this converter can perform the conversion; otherwise, false.
         /// </returns>
+        [ExcludeFromCodeCoverage]  // not called
         public override bool CanConvertTo(ITypeDescriptorContext context, Type destinationType)
         {
             return destinationType == typeof(string);

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
index 1ec7cab..eb454d6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
@@ -130,6 +130,7 @@ namespace Apache.Ignite.Core.Impl.Events
         }
 
         /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
         public Guid? RemoteListen<T>(int bufSize = 1, TimeSpan? interval = null, bool autoUnsubscribe = true,
             IEventFilter<T> localListener = null, IEventFilter<T> remoteFilter = null, params int[] types)
             where T : IEvent
@@ -163,6 +164,7 @@ namespace Apache.Ignite.Core.Impl.Events
         }
 
         /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
         public Guid? RemoteListen<T>(int bufSize = 1, TimeSpan? interval = null, bool autoUnsubscribe = true,
             IEventFilter<T> localListener = null, IEventFilter<T> remoteFilter = null, IEnumerable<int> types = null)
             where T : IEvent
@@ -171,6 +173,7 @@ namespace Apache.Ignite.Core.Impl.Events
         }
 
         /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
         public void StopRemoteListen(Guid opId)
         {
             DoOutOp((int) Op.StopRemoteListen, writer =>

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
index 461872f..a59ca5f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
@@ -205,6 +205,7 @@ namespace Apache.Ignite.Core.Impl
         /// <param name="msg">Message.</param>
         /// <param name="stackTrace">Stack trace.</param>
         /// <returns>Exception.</returns>
+        [ExcludeFromCodeCoverage]  // Covered by a test in a separate process.
         public static Exception GetJvmInitializeException(string clsName, string msg, string stackTrace)
         {
             if (clsName != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/Handle.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/Handle.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/Handle.cs
index fb56891..7791c91 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/Handle.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/Handle.cs
@@ -39,7 +39,7 @@ namespace Apache.Ignite.Core.Impl.Handle
         /// </summary>
         /// <param name="target">Target.</param>
         /// <param name="releaseAction">Release action.</param>
-        public Handle(T target, Action<T> releaseAction)
+        protected Handle(T target, Action<T> releaseAction)
         {
             _target = target;
             _releaseAction = releaseAction;
@@ -48,7 +48,7 @@ namespace Apache.Ignite.Core.Impl.Handle
         /// <summary>
         /// Target.
         /// </summary>
-        public T Target
+        protected T Target
         {
             get { return _target; }
         }
@@ -61,13 +61,5 @@ namespace Apache.Ignite.Core.Impl.Handle
             if (Interlocked.CompareExchange(ref _released, 1, 0) == 0)
                 _releaseAction(_target);
         }
-
-        /// <summary>
-        /// Resource released flag.
-        /// </summary>
-        public bool Released
-        {
-            get { return Thread.VolatileRead(ref _released) == 1; }
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/HandleRegistry.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/HandleRegistry.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/HandleRegistry.cs
index e18970f..4e1135a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/HandleRegistry.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/HandleRegistry.cs
@@ -102,16 +102,6 @@ namespace Apache.Ignite.Core.Impl.Handle
         }
 
         /// <summary>
-        /// Allocate a handle for critical resource in safe mode.
-        /// </summary>
-        /// <param name="target">Target.</param>
-        /// <returns>Pointer.</returns>
-        public long AllocateCriticalSafe(object target)
-        {
-            return Allocate0(target, true, true);
-        }
-
-        /// <summary>
         /// Internal allocation routine.
         /// </summary>
         /// <param name="target">Target.</param>
@@ -121,7 +111,7 @@ namespace Apache.Ignite.Core.Impl.Handle
         private long Allocate0(object target, bool critical, bool safe)
         {
             if (Closed)
-                throw ClosedException();
+                throw GetClosedException();
 
             // Try allocating on critical path.
             if (critical)
@@ -140,7 +130,7 @@ namespace Apache.Ignite.Core.Impl.Handle
 
                             Release0(target, true);
 
-                            throw ClosedException();
+                            throw GetClosedException();
                         }
 
                         return fastIdx;
@@ -159,7 +149,7 @@ namespace Apache.Ignite.Core.Impl.Handle
 
                 Release0(target, true);
 
-                throw ClosedException();
+                throw GetClosedException();
             }
 
             return slowIdx;
@@ -320,6 +310,7 @@ namespace Apache.Ignite.Core.Impl.Handle
         /// Gets a snapshot of currently referenced objects list.
         /// </summary>
         [SuppressMessage("Microsoft.Design", "CA1024:UsePropertiesWhereAppropriate", Justification = "Semantics.")]
+        [ExcludeFromCodeCoverage]
         public IList<KeyValuePair<long, object>> GetItems()
         {
             Thread.MemoryBarrier();
@@ -335,7 +326,8 @@ namespace Apache.Ignite.Core.Impl.Handle
         /// Create new exception for closed state.
         /// </summary>
         /// <returns>Exception.</returns>
-        private static Exception ClosedException()
+        [ExcludeFromCodeCoverage]
+        private static Exception GetClosedException()
         {
             return new InvalidOperationException("Cannot allocate a resource handle because Ignite is stopping.");
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/IHandle.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/IHandle.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/IHandle.cs
index d147f8b..700ab5f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/IHandle.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/IHandle.cs
@@ -26,10 +26,5 @@ namespace Apache.Ignite.Core.Impl.Handle
         /// Release the resource.
         /// </summary>
         void Release();
-
-        /// <summary>
-        /// Resource released flag.
-        /// </summary>
-        bool Released { get; }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
index 914a87d..8dc63bd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
@@ -41,6 +41,7 @@ namespace Apache.Ignite.Core.Impl
     /// Grid proxy with fake serialization.
     /// </summary>
     [Serializable]
+    [ExcludeFromCodeCoverage]
     internal class IgniteProxy : IIgnite, IClusterGroupEx, IBinaryWriteAware, ICluster
     {
         /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs
index 0a69e4a..de9daae 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs
@@ -20,7 +20,6 @@ namespace Apache.Ignite.Core.Impl
     using System;
     using System.Collections.Generic;
     using System.ComponentModel;
-    using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
     using System.Globalization;
     using System.IO;
@@ -238,6 +237,7 @@ namespace Apache.Ignite.Core.Impl
         /// <summary>
         /// Formats the Win32 error.
         /// </summary>
+        [ExcludeFromCodeCoverage]
         private static string FormatWin32Error(int errorCode)
         {
             if (errorCode == NativeMethods.ERROR_BAD_EXE_FORMAT)
@@ -493,25 +493,5 @@ namespace Apache.Ignite.Core.Impl
 
             return res;
         }
-
-        /// <summary>
-        /// Writes the node collection to a stream.
-        /// </summary>
-        /// <param name="writer">The writer.</param>
-        /// <param name="nodes">The nodes.</param>
-        public static void WriteNodes(IBinaryRawWriter writer, ICollection<IClusterNode> nodes)
-        {
-            Debug.Assert(writer != null);
-
-            if (nodes != null)
-            {
-                writer.WriteInt(nodes.Count);
-
-                foreach (var node in nodes)
-                    writer.WriteGuid(node.Id);
-            }
-            else
-                writer.WriteInt(-1);
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/InteropExceptionHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/InteropExceptionHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/InteropExceptionHolder.cs
deleted file mode 100644
index 9edcb03..0000000
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/InteropExceptionHolder.cs
+++ /dev/null
@@ -1,88 +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.
- */
-
-namespace Apache.Ignite.Core.Impl
-{
-    using System;
-    using System.Diagnostics.CodeAnalysis;
-    using System.Runtime.Serialization.Formatters.Binary;
-    using Apache.Ignite.Core.Binary;
-    using Apache.Ignite.Core.Impl.Binary;
-    using Apache.Ignite.Core.Impl.Binary.IO;
-
-    /// <summary>
-    /// Holder of exception which must be serialized to Java and then backwards to the native platform.
-    /// </summary>
-    internal class InteropExceptionHolder : IBinarizable
-    {
-        /** Initial exception. */
-        private readonly Exception _err;
-
-        /// <summary>
-        /// Constructor.
-        /// </summary>
-        public InteropExceptionHolder()
-        {
-            // No-op.
-        }
-
-        /// <summary>
-        /// Constructor.
-        /// </summary>
-        /// <param name="err">Error.</param>
-        public InteropExceptionHolder(Exception err)
-        {
-            _err = err;
-        }
-
-        /// <summary>
-        /// Underlying exception.
-        /// </summary>
-        public Exception Error
-        {
-            get { return _err; }
-        }
-
-        /** <inheritDoc /> */
-        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
-        public void WriteBinary(IBinaryWriter writer)
-        {
-            var writer0 = (BinaryWriter) writer.GetRawWriter();
-
-            if (writer0.IsBinarizable(_err))
-            {
-                writer0.WriteBoolean(true);
-                writer0.WriteObject(_err);
-            }
-            else
-            {
-                writer0.WriteBoolean(false);
-
-                using (var streamAdapter = new BinaryStreamAdapter(writer0.Stream))
-                {
-                    new BinaryFormatter().Serialize(streamAdapter, _err);
-                }
-            }
-        }
-
-        /** <inheritDoc /> */
-        public void ReadBinary(IBinaryReader reader)
-        {
-            throw new NotImplementedException();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs
index a991b3d..3aa5490 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryUtils.cs
@@ -88,16 +88,6 @@ namespace Apache.Ignite.Core.Impl.Memory
         }
 
         /// <summary>
-        /// Sets capacity for the given memory chunk.
-        /// </summary>
-        /// <param name="memPtr">Memory pointer.</param>
-        /// <param name="cap">CalculateCapacity.</param>
-        public static void SetCapacity(long memPtr, int cap) 
-        {
-            *((int*)(memPtr + MemHdrOffCap)) = cap;
-        }
-
-        /// <summary>
         /// Gets length for the given memory chunk.
         /// </summary>
         /// <param name="memPtr">Memory pointer.</param>
@@ -138,16 +128,6 @@ namespace Apache.Ignite.Core.Impl.Memory
         }
 
         /// <summary>
-        /// Check whether this memory chunk is external.
-        /// </summary>
-        /// <param name="memPtr">Memory pointer.</param>
-        /// <returns><c>True</c> if owned by Java.</returns>
-        public static bool IsExternal(long memPtr) 
-        {
-            return IsExternal(GetFlags(memPtr));
-        }
-
-        /// <summary>
         /// Check whether flags denote that this memory chunk is external.
         /// </summary>
         /// <param name="flags">Flags.</param>
@@ -158,16 +138,6 @@ namespace Apache.Ignite.Core.Impl.Memory
         }
 
         /// <summary>
-        /// Check whether this memory chunk is pooled.
-        /// </summary>
-        /// <param name="memPtr">Memory pointer.</param>
-        /// <returns><c>True</c> if pooled.</returns>
-        public static bool IsPooled(long memPtr) 
-        {
-            return IsPooled(GetFlags(memPtr));
-        }
-
-        /// <summary>
         /// Check whether flags denote pooled memory chunk.
         /// </summary>
         /// <param name="flags">Flags.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs
index 8e54261..f252ef3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs
@@ -52,6 +52,7 @@ namespace Apache.Ignite.Core.Impl.Memory
         }
 
         /** <inheritdoc /> */
+        [ExcludeFromCodeCoverage]
         public long Pointer
         {
             get { throw new NotSupportedException(); }
@@ -73,10 +74,13 @@ namespace Apache.Ignite.Core.Impl.Memory
         public int Length
         {
             get { return _size; }
+
+            [ExcludeFromCodeCoverage]
             set { throw new NotSupportedException(); }
         }
 
         /** <inheritdoc /> */
+        [ExcludeFromCodeCoverage]
         public void Reallocate(int cap)
         {
             throw new NotSupportedException();

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
index f741389..fb9d890 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
@@ -1032,13 +1032,5 @@ namespace Apache.Ignite.Core.Impl
             if (_disposed)
                 throw new ObjectDisposedException(GetType().Name, "Object has been disposed.");
         }
-
-        /// <summary>
-        /// Gets a value indicating whether this instance is disposed.
-        /// </summary>
-        protected bool IsDisposed
-        {
-            get { return _disposed; }
-        }
     }
 }


[37/50] [abbrv] ignite git commit: IGNITE-4185 .NET: Fix NullReferenceException in IgniteOutputCacheProvider when igniteConfiguration is missing

Posted by vo...@apache.org.
IGNITE-4185 .NET: Fix NullReferenceException in IgniteOutputCacheProvider when igniteConfiguration is missing


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

Branch: refs/heads/ignite-2693
Commit: d775ad12a8eedce5d3bd80c296de76b9ca2d49b7
Parents: 986e276
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Nov 8 19:13:48 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Wed Nov 9 19:53:44 2016 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.AspNet.Tests/App.config       |  1 +
 .../IgniteSessionStateStoreProviderTest.cs      | 21 +++++++++++++++++---
 .../Apache.Ignite.AspNet/Impl/ConfigUtil.cs     |  5 +++++
 3 files changed, 24 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d775ad12/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/App.config
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/App.config b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/App.config
index 86ee3d4..7d2c1d0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/App.config
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/App.config
@@ -22,6 +22,7 @@
         <section name="igniteConfiguration" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
         <section name="igniteConfiguration2" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
         <section name="igniteConfiguration3" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
+        <section name="igniteConfigurationInvalid" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
     </configSections>
 
     <runtime>

http://git-wip-us.apache.org/repos/asf/ignite/blob/d775ad12/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
index 2c73359..9c3b07c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
@@ -19,6 +19,7 @@ namespace Apache.Ignite.AspNet.Tests
 {
     using System;
     using System.Collections.Specialized;
+    using System.Configuration;
     using System.Linq;
     using System.Reflection;
     using System.Threading;
@@ -116,14 +117,28 @@ namespace Apache.Ignite.AspNet.Tests
             Assert.Throws<InvalidOperationException>(() =>
                     stateProvider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions));
 
-            // Invalid section.
-            Assert.Throws<IgniteException>(() =>
+            // Missing section.
+            var ex = Assert.Throws<IgniteException>(() =>
                 stateProvider.Initialize("testName", new NameValueCollection
                 {
-                    {SectionNameAttr, "invalidSection"},
+                    {SectionNameAttr, "missingSection"},
                     {CacheNameAttr, CacheName}
                 }));
 
+            Assert.IsInstanceOf<ConfigurationErrorsException>(ex.InnerException);
+
+            // Invalid section with missing content.
+            stateProvider = new IgniteSessionStateStoreProvider();
+
+            ex = Assert.Throws<IgniteException>(() =>
+                stateProvider.Initialize("testName", new NameValueCollection
+                {
+                    {SectionNameAttr, "igniteConfigurationInvalid"},
+                    {CacheNameAttr, CacheName}
+                }));
+
+            Assert.IsInstanceOf<ConfigurationErrorsException>(ex.InnerException);
+
             // Valid grid.
             stateProvider = GetProvider();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d775ad12/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs
index a162d81..fc93c7e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs
@@ -86,6 +86,11 @@ namespace Apache.Ignite.AspNet.Impl
                         "Could not find {0} with name '{1}'", typeof(IgniteConfigurationSection).Name, sectionName));
 
                 config = section.IgniteConfiguration;
+
+                if (config == null)
+                    throw new ConfigurationErrorsException(string.Format(CultureInfo.InvariantCulture,
+                        "{0} with name '{1}' is defined in <configSections>, but not present in configuration", 
+                        typeof(IgniteConfigurationSection).Name, sectionName));
             }
             else
                 config = new IgniteConfiguration {GridName = gridName};


[08/50] [abbrv] ignite git commit: IGNITE-3604 .NET: Fix inconsistent namespace imports in examples

Posted by vo...@apache.org.
IGNITE-3604 .NET: Fix inconsistent namespace imports in examples

Conflicts:
	modules/platforms/dotnet/Apache.Ignite.sln.DotSettings


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

Branch: refs/heads/ignite-2693
Commit: d7dee5248332c6c6276c9e21aed4afb3abdd4340
Parents: cd0275d
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Jul 29 15:40:42 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Wed Nov 2 11:15:55 2016 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.sln.DotSettings        |  4 +++
 .../Compute/ClosureExample.cs                   | 12 ++++----
 .../Compute/TaskExample.cs                      | 14 +++++-----
 .../Datagrid/ContinuousQueryExample.cs          | 16 +++++------
 .../Datagrid/DataStreamerExample.cs             | 12 ++++----
 .../Datagrid/LinqExample.cs                     | 29 ++++++++++----------
 .../Datagrid/PutGetExample.cs                   | 12 ++++----
 .../Datagrid/QueryExample.cs                    | 18 ++++++------
 .../Datagrid/StoreExample.cs                    | 20 +++++++-------
 .../Datagrid/TransactionExample.cs              | 14 +++++-----
 .../Events/EventsExample.cs                     | 16 +++++------
 .../Messaging/MessagingExample.cs               | 10 +++----
 .../Misc/LifecycleExample.cs                    | 13 ++++-----
 .../Services/IMapService.cs                     |  4 +--
 .../Services/ServicesExample.cs                 | 10 +++----
 .../Apache.Ignite.ExamplesDll/Binary/Account.cs |  6 ++--
 .../Apache.Ignite.ExamplesDll/Binary/Address.cs |  2 +-
 .../Binary/Employee.cs                          |  8 +++---
 .../Binary/EmployeeKey.cs                       |  4 +--
 .../Binary/Organization.cs                      |  2 +-
 .../Compute/AverageSalaryJob.cs                 | 11 ++++----
 .../Compute/AverageSalaryTask.cs                |  9 +++---
 .../Compute/CharacterCountClosure.cs            |  6 ++--
 .../Compute/CharacterCountReducer.cs            |  4 +--
 .../Datagrid/ContinuousQueryFilter.cs           |  6 ++--
 .../Datagrid/EmployeeStore.cs                   | 13 ++++-----
 .../Datagrid/EmployeeStorePredicate.cs          |  5 ++--
 .../Events/LocalListener.cs                     |  8 +++---
 .../Messaging/LocalListener.cs                  |  8 +++---
 .../Messaging/RemoteOrderedListener.cs          | 10 +++----
 .../Messaging/RemoteUnorderedListener.cs        | 10 +++----
 .../Services/MapService.cs                      | 12 ++++----
 32 files changed, 163 insertions(+), 165 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings b/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings
index 72ce015..078e9fb 100644
--- a/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings
+++ b/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings
@@ -1,5 +1,9 @@
 \ufeff<wpf:ResourceDictionary xml:space="preserve" xmlns:x="http://schemas.microsoft.com/winfx/2006/xaml" xmlns:s="clr-namespace:System;assembly=mscorlib" xmlns:ss="urn:shemas-jetbrains-com:settings-storage-xaml" xmlns:wpf="http://schemas.microsoft.com/winfx/2006/xaml/presentation">
 	<s:String x:Key="/Default/CodeInspection/CSharpLanguageProject/LanguageLevel/@EntryValue">CSharp50</s:String>
+	<s:String x:Key="/Default/CodeStyle/CodeCleanup/Profiles/=silent/@EntryIndexedValue">&lt;?xml version="1.0" encoding="utf-16"?&gt;&lt;Profile name="silent"&gt;&lt;CSReorderTypeMembers&gt;True&lt;/CSReorderTypeMembers&gt;&lt;CSCodeStyleAttributes ArrangeTypeAccessModifier="True" ArrangeTypeMemberAccessModifier="True" SortModifiers="True" RemoveRedundantParentheses="False" AddMissingParentheses="False" ArrangeBraces="False" ArrangeAttributes="False" ArrangeArgumentsStyle="False" /&gt;&lt;RemoveCodeRedundancies&gt;True&lt;/RemoveCodeRedundancies&gt;&lt;CSArrangeQualifiers&gt;True&lt;/CSArrangeQualifiers&gt;&lt;CSOptimizeUsings&gt;&lt;OptimizeUsings&gt;True&lt;/OptimizeUsings&gt;&lt;EmbraceInRegion&gt;False&lt;/EmbraceInRegion&gt;&lt;RegionName&gt;&lt;/RegionName&gt;&lt;/CSOptimizeUsings&gt;&lt;CSShortenReferences&gt;True&lt;/CSShortenReferences&gt;&lt;CSReformatCode&gt;True&lt;/CSReformatCode&gt;&lt;/Profile&gt;</s:String>
+	<s:String x:Key="/Default/CodeStyle/CodeCleanup/SilentCleanupProfile/@EntryValue">silent</s:String>
+	<s:String x:Key="/Default/CodeStyle/CodeFormatting/XmlDocFormatter/IndentSubtags/@EntryValue">DoNotTouch</s:String>
+	<s:String x:Key="/Default/CodeStyle/CodeFormatting/XmlDocFormatter/IndentTagContent/@EntryValue">DoNotTouch</s:String>
 	<s:Boolean x:Key="/Default/CodeStyle/CSharpUsing/AddImportsToDeepestScope/@EntryValue">True</s:Boolean>
 	<s:Boolean x:Key="/Default/CodeStyle/CSharpUsing/QualifiedUsingAtNestedScope/@EntryValue">True</s:Boolean>
 	<s:String x:Key="/Default/CodeInspection/Highlighting/InspectionSeverities/=ConvertClosureToMethodGroup/@EntryIndexedValue">DO_NOT_SHOW</s:String>

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Compute/ClosureExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Compute/ClosureExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Compute/ClosureExample.cs
index 34e32b9..0b5ac46 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Compute/ClosureExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Compute/ClosureExample.cs
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-using System;
-using System.Collections.Generic;
-using System.Linq;
-using Apache.Ignite.Core;
-using Apache.Ignite.ExamplesDll.Compute;
-
 namespace Apache.Ignite.Examples.Compute
 {
+    using System;
+    using System.Collections.Generic;
+    using System.Linq;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.ExamplesDll.Compute;
+
     /// <summary>
     /// Example demonstrating closure execution.
     /// <para />

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Compute/TaskExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Compute/TaskExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Compute/TaskExample.cs
index a7873cb..71e76a9 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Compute/TaskExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Compute/TaskExample.cs
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-using System;
-using System.Collections.Generic;
-using Apache.Ignite.Core;
-using Apache.Ignite.ExamplesDll.Compute;
-using Apache.Ignite.ExamplesDll.Binary;
-
 namespace Apache.Ignite.Examples.Compute
 {
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.ExamplesDll.Binary;
+    using Apache.Ignite.ExamplesDll.Compute;
+
     /// <summary>
     /// Example demonstrating task execution.
     /// <para />
@@ -78,7 +78,7 @@ namespace Apache.Ignite.Examples.Compute
         /// <returns>Collection of employees.</returns>
         private static ICollection<Employee> Employees()
         {
-            return new []
+            return new[]
             {
                 new Employee(
                     "James Wilson",

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/ContinuousQueryExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/ContinuousQueryExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/ContinuousQueryExample.cs
index e3fc583..d2e2726 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/ContinuousQueryExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/ContinuousQueryExample.cs
@@ -15,16 +15,16 @@
  * limitations under the License.
  */
 
-using System;
-using System.Collections.Generic;
-using System.Threading;
-using Apache.Ignite.Core;
-using Apache.Ignite.Core.Cache.Event;
-using Apache.Ignite.Core.Cache.Query.Continuous;
-using Apache.Ignite.ExamplesDll.Datagrid;
-
 namespace Apache.Ignite.Examples.Datagrid
 {
+    using System;
+    using System.Collections.Generic;
+    using System.Threading;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.Core.Cache.Event;
+    using Apache.Ignite.Core.Cache.Query.Continuous;
+    using Apache.Ignite.ExamplesDll.Datagrid;
+
     /// <summary>
     /// This example demonstrates continuous query API.
     /// <para />

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/DataStreamerExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/DataStreamerExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/DataStreamerExample.cs
index 88a0529..c4bf342 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/DataStreamerExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/DataStreamerExample.cs
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-using System;
-using System.Diagnostics;
-using Apache.Ignite.Core;
-using Apache.Ignite.Core.Datastream;
-using Apache.Ignite.ExamplesDll.Binary;
-
 namespace Apache.Ignite.Examples.Datagrid
 {
+    using System;
+    using System.Diagnostics;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.Core.Datastream;
+    using Apache.Ignite.ExamplesDll.Binary;
+
     /// <summary>
     /// Demonstrates how cache can be populated with data utilizing <see cref="IDataStreamer{TK,TV}"/>.
     /// Data streamer is a lot more efficient to use than standard cache put operation 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs
index b4a8493..2223600 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs
@@ -15,19 +15,18 @@
  * limitations under the License.
  */
 
-using System;
-using System.Linq;
-using System.Collections.Generic;
-
-using Apache.Ignite.Core;
-using Apache.Ignite.Linq;
-using Apache.Ignite.Core.Cache;
-using Apache.Ignite.Core.Cache.Configuration;
-using Apache.Ignite.Core.Cache.Query;
-using Apache.Ignite.ExamplesDll.Binary;
-
 namespace Apache.Ignite.Examples.Datagrid
 {
+    using System;
+    using System.Collections.Generic;
+    using System.Linq;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Cache.Query;
+    using Apache.Ignite.ExamplesDll.Binary;
+    using Apache.Ignite.Linq;
+
     /// <summary>
     /// This example populates cache with sample data and runs several LINQ queries over this data.
     /// <para />
@@ -103,7 +102,7 @@ namespace Apache.Ignite.Examples.Datagrid
         {
             const int zip = 94109;
 
-            IQueryable<ICacheEntry<EmployeeKey, Employee>> qry = 
+            IQueryable<ICacheEntry<EmployeeKey, Employee>> qry =
                 cache.AsCacheQueryable().Where(emp => emp.Value.Address.Zip == zip);
 
             Console.WriteLine();
@@ -122,7 +121,7 @@ namespace Apache.Ignite.Examples.Datagrid
             const int zip = 94109;
 
             // Compile cache query to eliminate LINQ overhead on multiple runs.
-            Func<int, IQueryCursor<ICacheEntry<EmployeeKey, Employee>>> qry = 
+            Func<int, IQueryCursor<ICacheEntry<EmployeeKey, Employee>>> qry =
                 CompiledQuery.Compile((int z) => cache.AsCacheQueryable().Where(emp => emp.Value.Address.Zip == z));
 
             Console.WriteLine();
@@ -137,7 +136,7 @@ namespace Apache.Ignite.Examples.Datagrid
         /// </summary>
         /// <param name="employeeCache">Employee cache.</param>
         /// <param name="organizationCache">Organization cache.</param>
-        private static void JoinQueryExample(ICache<EmployeeKey, Employee> employeeCache, 
+        private static void JoinQueryExample(ICache<EmployeeKey, Employee> employeeCache,
             ICache<int, Organization> organizationCache)
         {
             const string orgName = "Apache";
@@ -145,7 +144,7 @@ namespace Apache.Ignite.Examples.Datagrid
             IQueryable<ICacheEntry<EmployeeKey, Employee>> employees = employeeCache.AsCacheQueryable();
             IQueryable<ICacheEntry<int, Organization>> organizations = organizationCache.AsCacheQueryable();
 
-            IQueryable<ICacheEntry<EmployeeKey, Employee>> qry = 
+            IQueryable<ICacheEntry<EmployeeKey, Employee>> qry =
                 from employee in employees
                 from organization in organizations
                 where employee.Key.OrganizationId == organization.Key && organization.Value.Name == orgName

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/PutGetExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/PutGetExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/PutGetExample.cs
index ebf77ec..10a75fa 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/PutGetExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/PutGetExample.cs
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-using System;
-using System.Collections.Generic;
-using Apache.Ignite.Core;
-using Apache.Ignite.Core.Binary;
-using Apache.Ignite.ExamplesDll.Binary;
-
 namespace Apache.Ignite.Examples.Datagrid
 {
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.ExamplesDll.Binary;
+
     /// <summary>
     /// This example demonstrates several put-get operations on Ignite cache
     /// with binary values. Note that binary object can be retrieved in

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs
index 7880eac..ccd6fd9 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs
@@ -15,17 +15,17 @@
  * limitations under the License.
  */
 
-using System;
-using System.Collections;
-using System.Collections.Generic;
-using Apache.Ignite.Core;
-using Apache.Ignite.Core.Cache.Configuration;
-using Apache.Ignite.Core.Cache;
-using Apache.Ignite.Core.Cache.Query;
-using Apache.Ignite.ExamplesDll.Binary;
-
 namespace Apache.Ignite.Examples.Datagrid
 {
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Cache.Query;
+    using Apache.Ignite.ExamplesDll.Binary;
+
     /// <summary>
     /// This example populates cache with sample data and runs several SQL and
     /// full text queries over this data.

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/StoreExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/StoreExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/StoreExample.cs
index 25721b8..62da647 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/StoreExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/StoreExample.cs
@@ -15,15 +15,15 @@
  * limitations under the License.
  */
 
-using System;
-using System.Collections.Generic;
-using Apache.Ignite.Core;
-using Apache.Ignite.Core.Cache.Configuration;
-using Apache.Ignite.ExamplesDll.Binary;
-using Apache.Ignite.ExamplesDll.Datagrid;
-
 namespace Apache.Ignite.Examples.Datagrid
 {
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.ExamplesDll.Binary;
+    using Apache.Ignite.ExamplesDll.Datagrid;
+
     /// <summary>
     /// Example demonstrating cache store.
     /// <para />
@@ -38,7 +38,7 @@ namespace Apache.Ignite.Examples.Datagrid
     /// Apache.Ignite.exe -configFileName=platforms\dotnet\examples\apache.ignite.examples\app.config -assembly=[path_to_Apache.Ignite.ExamplesDll.dll]
     /// 2) Start example.
     /// </summary>
-    class StoreExample
+    public class StoreExample
     {
         /// <summary>Cache name.</summary>
         private const string CacheName = "dotnet_cache_with_store";
@@ -75,7 +75,7 @@ namespace Apache.Ignite.Examples.Datagrid
                 Console.WriteLine();
                 Console.WriteLine(">>> Loaded entry from store through ICache.LoadCache().");
                 Console.WriteLine(">>> Current cache size: " + cache.GetSize());
-                
+
                 // Load entry from store calling ICache.Get() method.
                 Employee emp = cache.Get(2);
 
@@ -97,7 +97,7 @@ namespace Apache.Ignite.Examples.Datagrid
 
                 // Clear values again.
                 cache.Clear();
-                
+
                 Console.WriteLine();
                 Console.WriteLine(">>> Cleared values from cache again.");
                 Console.WriteLine(">>> Current cache size: " + cache.GetSize());

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/TransactionExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/TransactionExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/TransactionExample.cs
index b0e953c..f90cf96 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/TransactionExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/TransactionExample.cs
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-using System;
-using Apache.Ignite.Core;
-using Apache.Ignite.Core.Cache.Configuration;
-using Apache.Ignite.ExamplesDll.Binary;
-using Apache.Ignite.Core.Transactions;
-
 namespace Apache.Ignite.Examples.Datagrid
 {
+    using System;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Transactions;
+    using Apache.Ignite.ExamplesDll.Binary;
+
     /// <summary>
     /// This example demonstrates how to use transactions on Apache cache.
     /// <para />
@@ -37,7 +37,7 @@ namespace Apache.Ignite.Examples.Datagrid
     /// Apache.Ignite.exe -configFileName=platforms\dotnet\examples\apache.ignite.examples\app.config -assembly=[path_to_Apache.Ignite.ExamplesDll.dll]
     /// 2) Start example.
     /// </summary>
-    class TransactionExample
+    public class TransactionExample
     {
         /// <summary>Cache name.</summary>
         private const string CacheName = "dotnet_cache_tx";

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Events/EventsExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Events/EventsExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Events/EventsExample.cs
index 3b804f6..938b132 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Events/EventsExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Events/EventsExample.cs
@@ -15,16 +15,16 @@
  * limitations under the License.
  */
 
-using System;
-using System.Linq;
-using Apache.Ignite.Core;
-using Apache.Ignite.Core.Events;
-using Apache.Ignite.ExamplesDll.Compute;
-using Apache.Ignite.ExamplesDll.Events;
-using Apache.Ignite.ExamplesDll.Binary;
-
 namespace Apache.Ignite.Examples.Events
 {
+    using System;
+    using System.Linq;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.Core.Events;
+    using Apache.Ignite.ExamplesDll.Binary;
+    using Apache.Ignite.ExamplesDll.Compute;
+    using Apache.Ignite.ExamplesDll.Events;
+
     /// <summary>
     /// Example demonstrating Ignite events.
     /// <para />

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Messaging/MessagingExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Messaging/MessagingExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Messaging/MessagingExample.cs
index be267df..6728e38 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Messaging/MessagingExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Messaging/MessagingExample.cs
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-using System;
-using System.Threading;
-using Apache.Ignite.Core;
-using Apache.Ignite.ExamplesDll.Messaging;
-
 namespace Apache.Ignite.Examples.Messaging
 {
+    using System;
+    using System.Threading;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.ExamplesDll.Messaging;
+
     /// <summary>
     /// Example demonstrating Ignite messaging. Should be run with standalone Apache Ignite.NET node.
     /// <para />

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Misc/LifecycleExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Misc/LifecycleExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Misc/LifecycleExample.cs
index b7acb61..4831b2c 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Misc/LifecycleExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Misc/LifecycleExample.cs
@@ -15,16 +15,15 @@
  * limitations under the License.
  */
 
-using System;
-using System.Collections.Generic;
-using Apache.Ignite.Core;
-using Apache.Ignite.Core.Lifecycle;
-using Apache.Ignite.Core.Resource;
-
 namespace Apache.Ignite.Examples.Misc
 {
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core;
     using Apache.Ignite.Core.Discovery.Tcp;
     using Apache.Ignite.Core.Discovery.Tcp.Static;
+    using Apache.Ignite.Core.Lifecycle;
+    using Apache.Ignite.Core.Resource;
 
     /// <summary>
     /// This example shows how to provide your own <see cref="ILifecycleBean"/> implementation
@@ -101,7 +100,7 @@ namespace Apache.Ignite.Examples.Misc
                 if (evt == LifecycleEventType.AfterNodeStart)
                     Started = true;
                 else if (evt == LifecycleEventType.AfterNodeStop)
-                    Started = false;          
+                    Started = false;
             }
 
             /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Services/IMapService.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Services/IMapService.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Services/IMapService.cs
index 7253a0b..4e2b883 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Services/IMapService.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Services/IMapService.cs
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-using Apache.Ignite.ExamplesDll.Services;
-
 namespace Apache.Ignite.Examples.Services
 {
+    using Apache.Ignite.ExamplesDll.Services;
+
     /// <summary>
     /// Interface for service proxy interaction.
     /// Actual service class (<see cref="MapService{TK,TV}"/>) does not have to implement this interface. 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Services/ServicesExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Services/ServicesExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Services/ServicesExample.cs
index fcfce14..245356d 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Services/ServicesExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Services/ServicesExample.cs
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-using System;
-using Apache.Ignite.Core;
-using Apache.Ignite.ExamplesDll.Services;
-
 namespace Apache.Ignite.Examples.Services
 {
+    using System;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.ExamplesDll.Services;
+
     /// <summary>
     /// Example demonstrating Ignite services.
     /// <para />
@@ -55,7 +55,7 @@ namespace Apache.Ignite.Examples.Services
 
                 // Get a sticky service proxy so that we will always be contacting the same remote node.
                 var prx = ignite.GetServices().GetServiceProxy<IMapService<int, string>>("service", true);
-                
+
                 for (var i = 0; i < 10; i++)
                     prx.Put(i, i.ToString());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Account.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Account.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Account.cs
index 4d3fc35..8d659e1 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Account.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Account.cs
@@ -34,12 +34,12 @@ namespace Apache.Ignite.ExamplesDll.Binary
             Id = id;
             Balance = balance;
         }
-    
+
         /// <summary>
         /// Account ID.
         /// </summary>
         public int Id { get; set; }
-    
+
         /// <summary>
         /// Account balance.
         /// </summary>
@@ -51,7 +51,7 @@ namespace Apache.Ignite.ExamplesDll.Binary
         /// <returns>
         /// A string that represents the current object.
         /// </returns>
-        override public String ToString()
+        public override String ToString()
         {
             return string.Format("{0} [id={1}, balance={2}]", typeof(Account).Name, Id, Balance);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Address.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Address.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Address.cs
index bb6ed7f..7053e78 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Address.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Address.cs
@@ -76,7 +76,7 @@ namespace Apache.Ignite.ExamplesDll.Binary
         /// <returns>
         /// A string that represents the current object.
         /// </returns>
-        override public string ToString()
+        public override string ToString()
         {
             return string.Format("{0} [street={1}, zip={2}]", typeof(Address).Name, Street, Zip);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs
index 44a3f11..0fc3230 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs
@@ -72,9 +72,9 @@ namespace Apache.Ignite.ExamplesDll.Binary
         /// <returns>
         /// A string that represents the current object.
         /// </returns>
-        override public string ToString()
+        public override string ToString()
         {
-            return string.Format("{0} [name={1}, salary={2}, address={3}, departments={4}]", typeof(Employee).Name, 
+            return string.Format("{0} [name={1}, salary={2}, address={3}, departments={4}]", typeof(Employee).Name,
                 Name, Salary, Address, CollectionToString(Departments));
         }
 
@@ -87,8 +87,8 @@ namespace Apache.Ignite.ExamplesDll.Binary
             if (col == null)
                 return "null";
 
-            var elements = col.Any() 
-                ? col.Select(x => x.ToString()).Aggregate((x, y) => x + ", " + y) 
+            var elements = col.Any()
+                ? col.Select(x => x.ToString()).Aggregate((x, y) => x + ", " + y)
                 : string.Empty;
 
             return string.Format("[{0}]", elements);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/EmployeeKey.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/EmployeeKey.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/EmployeeKey.cs
index 9c1d8cc..e699648 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/EmployeeKey.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/EmployeeKey.cs
@@ -47,7 +47,7 @@ namespace Apache.Ignite.ExamplesDll.Binary
         [AffinityKeyMapped]
         [QuerySqlField(IsIndexed = true)]
         public int OrganizationId { get; private set; }
-        
+
         /// <summary>
         /// Determines whether the specified <see cref="T:System.Object"/> is equal to the current <see cref="T:System.Object"/>.
         /// </summary>
@@ -82,7 +82,7 @@ namespace Apache.Ignite.ExamplesDll.Binary
         /// </returns>
         public override string ToString()
         {
-            return string.Format("{0} [id={1}, organizationId={2}]", typeof (EmployeeKey).Name, Id, OrganizationId);
+            return string.Format("{0} [id={1}, organizationId={2}]", typeof(EmployeeKey).Name, Id, OrganizationId);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Organization.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Organization.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Organization.cs
index 16d4113..7495ee9 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Organization.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Organization.cs
@@ -70,7 +70,7 @@ namespace Apache.Ignite.ExamplesDll.Binary
         /// <filterpriority>2</filterpriority>
         public override string ToString()
         {
-            return string.Format("{0} [name={1}, address={2}, type={3}, lastUpdated={4}]", typeof (Organization).Name,
+            return string.Format("{0} [name={1}, address={2}, type={3}, lastUpdated={4}]", typeof(Organization).Name,
                 Name, Address, Type, LastUpdated);
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs
index e05a436..2685df0 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs
@@ -15,12 +15,11 @@
  * limitations under the License.
  */
 
-using System;
-using System.Collections.Generic;
-using Apache.Ignite.Core.Compute;
-
 namespace Apache.Ignite.ExamplesDll.Compute
 {
+    using System;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Compute;
     using Apache.Ignite.ExamplesDll.Binary;
 
     /// <summary>
@@ -33,7 +32,7 @@ namespace Apache.Ignite.ExamplesDll.Compute
         private readonly ICollection<Employee> _employees = new List<Employee>();
 
         /// <summary>
-        /// Adds employee.
+        ///     Adds employee.
         /// </summary>
         /// <param name="employee">Employee.</param>
         public void Add(Employee employee)
@@ -45,7 +44,7 @@ namespace Apache.Ignite.ExamplesDll.Compute
         /// Execute the job.
         /// </summary>
         /// <returns>Job result: tuple with total salary in the first item and employees count in the second.</returns>
-        override public Tuple<long, int> Execute()
+        public override Tuple<long, int> Execute()
         {
             long sum = 0;
             int count = 0;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryTask.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryTask.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryTask.cs
index 3af166f..6ef9798 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryTask.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryTask.cs
@@ -15,13 +15,12 @@
  * limitations under the License.
  */
 
-using System;
-using System.Collections.Generic;
-using System.Linq;
-using Apache.Ignite.Core.Compute;
-
 namespace Apache.Ignite.ExamplesDll.Compute
 {
+    using System;
+    using System.Collections.Generic;
+    using System.Linq;
+    using Apache.Ignite.Core.Compute;
     using Apache.Ignite.ExamplesDll.Binary;
 
     /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/CharacterCountClosure.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/CharacterCountClosure.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/CharacterCountClosure.cs
index 2823221..a34c3fa 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/CharacterCountClosure.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/CharacterCountClosure.cs
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-using System;
-using Apache.Ignite.Core.Compute;
-
 namespace Apache.Ignite.ExamplesDll.Compute
 {
+    using System;
+    using Apache.Ignite.Core.Compute;
+
     /// <summary>
     /// Closure counting characters in a string.
     /// </summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/CharacterCountReducer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/CharacterCountReducer.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/CharacterCountReducer.cs
index 6825046..9ebf7eb 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/CharacterCountReducer.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/CharacterCountReducer.cs
@@ -15,10 +15,10 @@
  * limitations under the License.
  */
 
-using Apache.Ignite.Core.Compute;
-
 namespace Apache.Ignite.ExamplesDll.Compute
 {
+    using Apache.Ignite.Core.Compute;
+
     /// <summary>
     /// Character count reducer which collects individual string lengths and aggregate them.
     /// </summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/ContinuousQueryFilter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/ContinuousQueryFilter.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/ContinuousQueryFilter.cs
index 8c05f42..f569f3e 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/ContinuousQueryFilter.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/ContinuousQueryFilter.cs
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-using System;
-using Apache.Ignite.Core.Cache.Event;
-
 namespace Apache.Ignite.ExamplesDll.Datagrid
 {
+    using System;
+    using Apache.Ignite.Core.Cache.Event;
+
     /// <summary>
     /// Filter for continuous query example.
     /// </summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/EmployeeStore.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/EmployeeStore.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/EmployeeStore.cs
index 561d83f..7049011 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/EmployeeStore.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/EmployeeStore.cs
@@ -15,15 +15,14 @@
  * limitations under the License.
  */
 
-using System;
-using System.Collections;
-using System.Collections.Concurrent;
-using System.Collections.Generic;
-using Apache.Ignite.Core.Cache;
-using Apache.Ignite.Core.Cache.Store;
-
 namespace Apache.Ignite.ExamplesDll.Datagrid
 {
+    using System;
+    using System.Collections;
+    using System.Collections.Concurrent;
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Store;
     using Apache.Ignite.ExamplesDll.Binary;
 
     /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/EmployeeStorePredicate.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/EmployeeStorePredicate.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/EmployeeStorePredicate.cs
index c25b2fa..5c3bf89 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/EmployeeStorePredicate.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/EmployeeStorePredicate.cs
@@ -15,11 +15,10 @@
  * limitations under the License.
  */
 
-using System;
-using Apache.Ignite.Core.Cache;
-
 namespace Apache.Ignite.ExamplesDll.Datagrid
 {
+    using System;
+    using Apache.Ignite.Core.Cache;
     using Apache.Ignite.ExamplesDll.Binary;
 
     /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/LocalListener.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/LocalListener.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/LocalListener.cs
index 46524a1..77af7a9 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/LocalListener.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Events/LocalListener.cs
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-using System;
-using System.Threading;
-using Apache.Ignite.Core.Events;
-
 namespace Apache.Ignite.ExamplesDll.Events
 {
+    using System;
+    using System.Threading;
+    using Apache.Ignite.Core.Events;
+
     /// <summary>
     /// Local event listener.
     /// </summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/LocalListener.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/LocalListener.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/LocalListener.cs
index 591d426..e8b2f4b 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/LocalListener.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/LocalListener.cs
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-using System;
-using System.Threading;
-using Apache.Ignite.Core.Messaging;
-
 namespace Apache.Ignite.ExamplesDll.Messaging
 {
+    using System;
+    using System.Threading;
+    using Apache.Ignite.Core.Messaging;
+
     /// <summary>
     /// Local message listener which signals countdown event on each received message.
     /// </summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/RemoteOrderedListener.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/RemoteOrderedListener.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/RemoteOrderedListener.cs
index 85538c2..edf38f2 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/RemoteOrderedListener.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/RemoteOrderedListener.cs
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-using System;
-using Apache.Ignite.Core;
-using Apache.Ignite.Core.Messaging;
-using Apache.Ignite.Core.Resource;
-
 namespace Apache.Ignite.ExamplesDll.Messaging
 {
+    using System;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.Core.Messaging;
+    using Apache.Ignite.Core.Resource;
+
     /// <summary>
     /// Listener for Ordered topic.
     /// </summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/RemoteUnorderedListener.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/RemoteUnorderedListener.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/RemoteUnorderedListener.cs
index ab23e8b..8054d36 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/RemoteUnorderedListener.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Messaging/RemoteUnorderedListener.cs
@@ -15,13 +15,13 @@
  * limitations under the License.
  */
 
-using System;
-using Apache.Ignite.Core;
-using Apache.Ignite.Core.Messaging;
-using Apache.Ignite.Core.Resource;
-
 namespace Apache.Ignite.ExamplesDll.Messaging
 {
+    using System;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.Core.Messaging;
+    using Apache.Ignite.Core.Resource;
+
     /// <summary>
     /// Listener for Unordered topic.
     /// </summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7dee524/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Services/MapService.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Services/MapService.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Services/MapService.cs
index d577ff7..958d91d 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Services/MapService.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Services/MapService.cs
@@ -15,14 +15,14 @@
  * limitations under the License.
  */
 
-using System;
-using Apache.Ignite.Core;
-using Apache.Ignite.Core.Cache;
-using Apache.Ignite.Core.Resource;
-using Apache.Ignite.Core.Services;
-
 namespace Apache.Ignite.ExamplesDll.Services
 {
+    using System;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Resource;
+    using Apache.Ignite.Core.Services;
+
     /// <summary>
     /// Service implementation.
     /// </summary>


[13/50] [abbrv] ignite git commit: IGNITE-4123 .NET: Remove [Serializable] from Employee in examples

Posted by vo...@apache.org.
IGNITE-4123 .NET: Remove [Serializable] from Employee in examples


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

Branch: refs/heads/ignite-2693
Commit: 74f8308d10fc011c00e52efcdb315b35cc79e60a
Parents: d5e15af
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Wed Nov 2 15:59:15 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Wed Nov 2 15:59:15 2016 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs | 6 ------
 .../dotnet/examples/Apache.Ignite.Examples/App.config          | 1 +
 .../examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs      | 3 +--
 .../Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs      | 5 ++---
 4 files changed, 4 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/74f8308d/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs
index 8b67462..1a2c18a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessageListenerHolder.cs
@@ -102,12 +102,6 @@ namespace Apache.Ignite.Core.Impl.Messaging
                 DestroyAction();
         }
 
-        /** <inheritDoc /> */
-        public bool Released
-        {
-            get { return false; } // Multiple releases are allowed.
-        }
-
         /// <summary>
         /// Creates local holder instance.
         /// </summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f8308d/modules/platforms/dotnet/examples/Apache.Ignite.Examples/App.config
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/App.config b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/App.config
index 1d180e8..8b16df3 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/App.config
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/App.config
@@ -37,6 +37,7 @@
                 <string>Apache.Ignite.ExamplesDll.Binary.Employee</string>
                 <string>Apache.Ignite.ExamplesDll.Binary.EmployeeKey</string>
                 <string>Apache.Ignite.ExamplesDll.Binary.Organization</string>
+                <string>Apache.Ignite.ExamplesDll.Compute.AverageSalaryJob</string>
             </types>
         </binaryConfiguration>
         

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f8308d/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs
index 4cff2a8..b746bdf 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs
@@ -17,7 +17,6 @@
 
 namespace Apache.Ignite.ExamplesDll.Binary
 {
-    using System;
     using System.Collections.Generic;
     using System.Linq;
     using Apache.Ignite.Core.Cache.Configuration;
@@ -25,7 +24,6 @@ namespace Apache.Ignite.ExamplesDll.Binary
     /// <summary>
     /// Employee.
     /// </summary>
-    [Serializable]
     public class Employee
     {
         /// <summary>
@@ -35,6 +33,7 @@ namespace Apache.Ignite.ExamplesDll.Binary
         /// <param name="salary">Salary.</param>
         /// <param name="address">Address.</param>
         /// <param name="departments">Departments.</param>
+        /// <param name="organizationId">The organization identifier.</param>
         public Employee(string name, long salary, Address address, ICollection<string> departments, 
             int organizationId = 0)
         {

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f8308d/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs
index 2685df0..491f3c1 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Compute/AverageSalaryJob.cs
@@ -18,18 +18,17 @@
 namespace Apache.Ignite.ExamplesDll.Compute
 {
     using System;
-    using System.Collections.Generic;
+    using System.Collections;
     using Apache.Ignite.Core.Compute;
     using Apache.Ignite.ExamplesDll.Binary;
 
     /// <summary>
     /// Average salary job.
     /// </summary>
-    [Serializable]
     public class AverageSalaryJob : ComputeJobAdapter<Tuple<long, int>>
     {
         /// <summary> Employees. </summary>
-        private readonly ICollection<Employee> _employees = new List<Employee>();
+        private readonly ArrayList _employees = new ArrayList();
 
         /// <summary>
         ///     Adds employee.


[18/50] [abbrv] ignite git commit: Merge branch 'ignite-1.7.3' into ignite-1.7.4

Posted by vo...@apache.org.
Merge branch 'ignite-1.7.3' into ignite-1.7.4


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

Branch: refs/heads/ignite-2693
Commit: 348593986b56ddfcec4a4455e49d9b279eae4dc8
Parents: 74f8308 a62a013
Author: devozerov <vo...@gridgain.com>
Authored: Sat Nov 5 13:28:03 2016 +0300
Committer: thatcoach <pp...@list.ru>
Committed: Sat Nov 5 13:29:08 2016 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteScheduler.java |   13 +
 .../cache/query/QueryCancelledException.java    |   35 +
 .../apache/ignite/cache/query/QueryCursor.java  |    8 +-
 .../ignite/cache/query/SqlFieldsQuery.java      |   26 +
 .../org/apache/ignite/cache/query/SqlQuery.java |   25 +
 .../ignite/internal/IgniteSchedulerImpl.java    |   18 +
 .../processors/cache/QueryCursorImpl.java       |   92 +-
 .../internal/processors/job/GridJobWorker.java  |   10 +-
 .../processors/query/GridQueryCancel.java       |   83 +
 .../processors/query/GridQueryFieldsResult.java |    3 +-
 .../query/GridQueryFieldsResultAdapter.java     |    3 +-
 .../processors/query/GridQueryIndexing.java     |   11 +-
 .../processors/query/GridQueryProcessor.java    |  105 +-
 .../twostep/messages/GridQueryFailResponse.java |   36 +-
 .../h2/twostep/messages/GridQueryRequest.java   |   31 +-
 .../service/GridServiceProcessor.java           |   61 +-
 .../internal/util/SerializableTransient.java    |   58 +
 .../util/nio/ssl/BlockingSslHandler.java        |   50 +-
 .../ignite/marshaller/MarshallerUtils.java      |   22 +
 .../optimized/OptimizedClassDescriptor.java     |   90 +-
 ...CommunicationSpiSslSmallBuffersSelfTest.java |   43 +
 .../IgniteSpiCommunicationSelfTestSuite.java    |    7 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  160 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |   66 +-
 .../query/h2/twostep/GridMergeIndex.java        |   12 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  117 +-
 .../h2/twostep/msg/GridH2QueryRequest.java      |   42 +-
 ...niteCacheDistributedQueryCancelSelfTest.java |  176 +
 ...butedQueryStopOnCancelOrTimeoutSelfTest.java |  253 ++
 ...cheQueryAbstractDistributedJoinSelfTest.java |  290 ++
 ...QueryNodeRestartDistributedJoinSelfTest.java |  262 +-
 .../IgniteCacheQueryNodeRestartSelfTest2.java   |  125 +-
 ...nCancelOrTimeoutDistributedJoinSelfTest.java |  138 +
 ...eCacheLocalQueryCancelOrTimeoutSelfTest.java |  158 +
 .../h2/GridIndexingSpiAbstractSelfTest.java     |    4 +-
 .../IgniteCacheQuerySelfTestSuite.java          |    4 +-
 .../IgniteCacheQuerySelfTestSuite2.java         |   10 +
 modules/web-console/DEVNOTES.txt                |    2 +-
 modules/web-console/backend/app/agent.js        |   26 +-
 modules/web-console/backend/app/browser.js      |   14 +-
 modules/web-console/backend/app/mongo.js        |  207 +-
 .../backend/routes/demo/domains.json            |   15 +-
 modules/web-console/backend/services/agents.js  |    1 +
 modules/web-console/backend/services/auth.js    |    8 +-
 modules/web-console/backend/services/caches.js  |   14 +-
 .../web-console/backend/services/clusters.js    |   19 +-
 modules/web-console/backend/services/domains.js |   12 +-
 modules/web-console/backend/services/igfss.js   |   12 +-
 .../backend/test/unit/AuthService.test.js       |    4 +-
 .../backend/test/unit/CacheService.test.js      |  135 +-
 .../backend/test/unit/ClusterService.test.js    |  132 +-
 .../backend/test/unit/DomainService.test.js     |  144 +-
 .../backend/test/unit/IgfsService.test.js       |  122 +-
 modules/web-console/frontend/.eslintrc          |    6 +-
 modules/web-console/frontend/app/app.config.js  |   10 +
 modules/web-console/frontend/app/app.js         |   37 +-
 .../web-console/frontend/app/data/dialects.json |    9 +
 .../frontend/app/data/java-classes.json         |    4 +-
 .../frontend/app/decorator/tooltip.js           |   13 +-
 .../app/directives/on-focus-out.directive.js    |   37 +
 .../directives/restore-input-focus.directive.js |   24 +
 .../ui-ace-java/ui-ace-java.controller.js       |   78 +-
 .../ui-ace-java/ui-ace-java.directive.js        |  100 +-
 .../ui-ace-pojos/ui-ace-pojos.controller.js     |    2 +-
 .../ui-ace-pom/ui-ace-pom.controller.js         |    2 +-
 .../ui-ace-sharp/ui-ace-sharp.controller.js     |   32 +
 .../ui-ace-sharp/ui-ace-sharp.directive.js      |  133 +
 .../directives/ui-ace-sharp/ui-ace-sharp.jade   |   22 +
 .../ui-ace-spring/ui-ace-spring.controller.js   |   88 +
 .../ui-ace-spring/ui-ace-spring.directive.js    |   66 +
 .../directives/ui-ace-spring/ui-ace-spring.jade |   17 +
 .../ui-ace-xml/ui-ace-xml.controller.js         |   27 -
 .../ui-ace-xml/ui-ace-xml.directive.js          |  147 -
 .../app/directives/ui-ace-xml/ui-ace-xml.jade   |   17 -
 .../ui-grid-settings/ui-grid-settings.jade      |   33 +
 .../ui-grid-settings/ui-grid-settings.scss      |   38 +
 .../frontend/app/filters/default-name.filter.js |   21 +
 .../frontend/app/filters/hasPojo.filter.js      |    5 +-
 .../helpers/jade/form/form-field-feedback.jade  |    5 +-
 .../frontend/app/helpers/jade/mixins.jade       |  219 +-
 .../frontend/app/modules/agent/agent.module.js  |   10 +-
 .../modules/configuration/Version.service.js    |   82 +-
 .../configuration/configuration.module.js       |   30 +-
 .../generator/AbstractTransformer.js            |  341 ++
 .../modules/configuration/generator/Beans.js    |  379 ++
 .../generator/ConfigurationGenerator.js         | 1785 +++++++++
 .../configuration/generator/Java.service.js     |   21 -
 .../generator/JavaTransformer.service.js        | 1721 +++++++++
 .../generator/PlatformGenerator.js              |  522 +++
 .../configuration/generator/Pom.service.js      |  173 +-
 .../generator/Properties.service.js             |   74 +
 .../configuration/generator/Readme.service.js   |   79 +
 .../generator/SharpTransformer.service.js       |  243 ++
 .../generator/SpringTransformer.service.js      |  325 ++
 .../configuration/generator/StringBuilder.js    |   76 +
 .../configuration/generator/Xml.service.js      |   21 -
 .../defaults/cache.platform.provider.js         |   60 +
 .../generator/defaults/cache.provider.js        |  129 +
 .../defaults/cluster.platform.provider.js       |   49 +
 .../generator/defaults/cluster.provider.js      |  293 ++
 .../generator/defaults/igfs.provider.js         |   68 +
 .../configuration/generator/generator-common.js |  625 +++
 .../configuration/generator/generator-java.js   | 3617 ++++++++++++++++++
 .../generator/generator-optional.js             |   25 +
 .../configuration/generator/generator-spring.js | 2111 ++++++++++
 .../app/modules/form/field/input/select.scss    |   21 -
 .../app/modules/form/field/input/text.scss      |    1 -
 .../frontend/app/modules/form/form.module.js    |   10 +-
 .../app/modules/form/panel/revert.directive.js  |    4 +-
 .../modules/form/services/FormGUID.service.js   |   22 +
 .../validator/java-built-in-class.directive.js  |    6 +-
 .../form/validator/java-identifier.directive.js |    6 +-
 .../form/validator/java-keywords.directive.js   |   15 +-
 .../java-package-specified.directive.js         |    6 +-
 .../frontend/app/modules/nodes/Nodes.service.js |   69 +
 .../modules/nodes/nodes-dialog.controller.js    |   68 +
 .../app/modules/nodes/nodes-dialog.jade         |   35 +
 .../app/modules/nodes/nodes-dialog.scss         |   37 +
 .../frontend/app/modules/nodes/nodes.module.js  |   27 +
 .../frontend/app/modules/sql/sql.controller.js  |  269 +-
 .../app/modules/states/configuration.state.js   |    2 +-
 .../configuration/caches/client-near-cache.jade |   50 +
 .../states/configuration/caches/general.jade    |    3 +
 .../states/configuration/caches/memory.jade     |   14 +-
 .../configuration/caches/near-cache-client.jade |   51 +
 .../configuration/caches/near-cache-server.jade |   52 +
 .../configuration/caches/node-filter.jade       |   49 -
 .../states/configuration/caches/query.jade      |   30 +-
 .../states/configuration/caches/rebalance.jade  |    3 +-
 .../configuration/caches/server-near-cache.jade |   51 -
 .../states/configuration/caches/store.jade      |   84 +-
 .../states/configuration/clusters/atomic.jade   |    3 +-
 .../configuration/clusters/attributes.jade      |    4 +-
 .../states/configuration/clusters/binary.jade   |   28 +-
 .../configuration/clusters/cache-key-cfg.jade   |    9 +-
 .../configuration/clusters/checkpoint.jade      |   85 +
 .../configuration/clusters/checkpoint/fs.jade   |   66 +
 .../configuration/clusters/checkpoint/jdbc.jade |   45 +
 .../configuration/clusters/checkpoint/s3.jade   |  174 +
 .../configuration/clusters/collision.jade       |   13 +-
 .../clusters/collision/job-stealing.jade        |    2 +-
 .../configuration/clusters/communication.jade   |    3 +-
 .../configuration/clusters/connector.jade       |    3 +-
 .../configuration/clusters/deployment.jade      |    3 +-
 .../configuration/clusters/discovery.jade       |    3 +-
 .../states/configuration/clusters/events.jade   |   37 +-
 .../states/configuration/clusters/failover.jade |   19 +-
 .../states/configuration/clusters/general.jade  |    3 +
 .../clusters/general/discovery/zookeeper.jade   |    2 +
 .../states/configuration/clusters/igfs.jade     |    3 +-
 .../configuration/clusters/load-balancing.jade  |  104 +
 .../configuration/clusters/marshaller.jade      |    3 +-
 .../states/configuration/clusters/odbc.jade     |    3 +-
 .../states/configuration/clusters/ssl.jade      |    7 +-
 .../states/configuration/clusters/swap.jade     |    3 +-
 .../configuration/clusters/transactions.jade    |    6 +-
 .../states/configuration/domains/general.jade   |   12 +-
 .../states/configuration/domains/query.jade     |   40 +-
 .../states/configuration/domains/store.jade     |   15 +-
 .../states/configuration/igfs/general.jade      |    3 +
 .../modules/states/configuration/igfs/ipc.jade  |    2 +-
 .../modules/states/configuration/igfs/misc.jade |    4 +-
 .../states/configuration/igfs/secondary.jade    |    3 +-
 .../configuration/summary/summary.controller.js |  106 +-
 .../app/services/ErrorPopover.service.js        |    2 +-
 .../frontend/app/services/FormUtils.service.js  |    6 +-
 .../frontend/app/services/JavaTypes.service.js  |   70 +-
 .../app/services/LegacyTable.service.js         |   24 +-
 .../app/services/LegacyUtils.service.js         |  128 +-
 .../frontend/app/services/SqlTypes.service.js   |    6 +-
 modules/web-console/frontend/app/vendor.js      |    3 +-
 .../frontend/controllers/caches-controller.js   |  121 +-
 .../frontend/controllers/clusters-controller.js |  177 +-
 .../frontend/controllers/domains-controller.js  |  148 +-
 .../frontend/controllers/igfs-controller.js     |    6 +-
 .../frontend/generator/generator-common.js      |  625 ---
 .../frontend/generator/generator-java.js        | 3611 -----------------
 .../frontend/generator/generator-optional.js    |   25 -
 .../frontend/generator/generator-properties.js  |  175 -
 .../frontend/generator/generator-readme.js      |   85 -
 .../frontend/generator/generator-xml.js         | 2108 ----------
 .../frontend/gulpfile.babel.js/paths.js         |   31 +-
 .../frontend/gulpfile.babel.js/tasks/jade.js    |   21 +-
 .../frontend/gulpfile.babel.js/tasks/watch.js   |    6 +-
 .../gulpfile.babel.js/webpack/common.js         |    2 +-
 .../webpack/environments/development.js         |   17 +-
 .../webpack/environments/test.js                |   52 +
 .../frontend/gulpfile.babel.js/webpack/index.js |    4 +-
 modules/web-console/frontend/package.json       |   16 +-
 .../frontend/public/stylesheets/style.scss      |   75 +-
 .../frontend/test/karma.conf.babel.js           |   91 +
 modules/web-console/frontend/test/karma.conf.js |   98 +-
 .../frontend/test/unit/JavaTransformer.test.js  |   57 +
 .../frontend/test/unit/JavaTypes.test.js        |   23 +-
 .../frontend/test/unit/SharpTransformer.test.js |   55 +
 .../test/unit/SpringTransformer.test.js         |   57 +
 .../frontend/test/unit/SqlTypes.test.js         |   17 -
 .../frontend/test/unit/Version.test.js          |   82 +
 .../test/unit/defaultName.filter.test.js        |   38 +
 .../frontend/views/configuration/caches.jade    |    3 +-
 .../frontend/views/configuration/clusters.jade  |    3 +-
 .../views/configuration/domains-import.jade     |   14 +-
 .../summary-project-structure.jade              |    2 +-
 .../frontend/views/configuration/summary.jade   |   57 +-
 .../frontend/views/sql/cache-metadata.jade      |    2 +-
 .../frontend/views/sql/notebook-new.jade        |    4 +-
 modules/web-console/frontend/views/sql/sql.jade |   80 +-
 .../views/templates/agent-download.jade         |    6 +-
 .../frontend/views/templates/batch-confirm.jade |    4 +-
 .../frontend/views/templates/clone.jade         |    4 +-
 .../frontend/views/templates/confirm.jade       |    4 +-
 .../frontend/views/templates/demo-info.jade     |    4 +-
 .../views/templates/getting-started.jade        |    4 +-
 .../frontend/views/templates/message.jade       |    4 +-
 .../frontend/views/templates/select.jade        |    2 +-
 .../ignite/console/demo/AgentClusterDemo.java   |    2 +-
 216 files changed, 18733 insertions(+), 8894 deletions(-)
----------------------------------------------------------------------



[10/50] [abbrv] ignite git commit: IGNITE-3673 .NET: Add examples for distributed joins.

Posted by vo...@apache.org.
IGNITE-3673 .NET: Add examples for distributed joins.


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

Branch: refs/heads/ignite-2693
Commit: 861c1736facdf76c71e93ddada0d8e15b5b2950c
Parents: 390c8d5
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Thu Aug 11 17:32:13 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Wed Nov 2 11:18:02 2016 +0300

----------------------------------------------------------------------
 .../Datagrid/LinqExample.cs                     | 195 +++++++++++++------
 .../Datagrid/QueryExample.cs                    | 174 ++++++++++++-----
 .../Apache.Ignite.ExamplesDll.csproj            |   1 -
 .../Binary/Employee.cs                          |  10 +-
 .../Binary/EmployeeKey.cs                       |  88 ---------
 5 files changed, 271 insertions(+), 197 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/861c1736/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs
index 848d8f5..86739b4 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs
@@ -18,10 +18,10 @@
 namespace Apache.Ignite.Examples.Datagrid
 {
     using System;
-    using System.Collections.Generic;
     using System.Linq;
     using Apache.Ignite.Core;
     using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Affinity;
     using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Cache.Query;
     using Apache.Ignite.ExamplesDll.Binary;
@@ -49,6 +49,9 @@ namespace Apache.Ignite.Examples.Datagrid
         /// <summary>Employee cache name.</summary>
         private const string EmployeeCacheName = "dotnet_cache_query_employee";
 
+        /// <summary>Colocated employee cache name.</summary>
+        private const string EmployeeCacheNameColocated = "dotnet_cache_query_employee_colocated";
+
         [STAThread]
         public static void Main()
         {
@@ -57,26 +60,18 @@ namespace Apache.Ignite.Examples.Datagrid
                 Console.WriteLine();
                 Console.WriteLine(">>> Cache LINQ example started.");
 
-                var employeeCache = ignite.GetOrCreateCache<EmployeeKey, Employee>(new CacheConfiguration
-                {
-                    Name = EmployeeCacheName,
-                    QueryEntities = new[]
-                    {
-                        new QueryEntity(typeof(EmployeeKey), typeof(Employee))
-                    }
-                });
-
-                var organizationCache = ignite.GetOrCreateCache<int, Organization>(new CacheConfiguration
-                {
-                    Name = OrganizationCacheName,
-                    QueryEntities = new[]
-                    {
-                        new QueryEntity(typeof(int), typeof(Organization))
-                    }
-                });
+                var employeeCache = ignite.GetOrCreateCache<int, Employee>(
+                    new CacheConfiguration(EmployeeCacheName, typeof(Employee)));
+
+                var employeeCacheColocated = ignite.GetOrCreateCache<AffinityKey, Employee>(
+                    new CacheConfiguration(EmployeeCacheNameColocated, typeof(Employee)));
+
+                var organizationCache = ignite.GetOrCreateCache<int, Organization>(
+                    new CacheConfiguration(OrganizationCacheName, new QueryEntity(typeof(int), typeof(Organization))));
 
                 // Populate cache with sample data entries.
                 PopulateCache(employeeCache);
+                PopulateCache(employeeCacheColocated);
                 PopulateCache(organizationCache);
 
                 // Run SQL query example.
@@ -86,7 +81,10 @@ namespace Apache.Ignite.Examples.Datagrid
                 CompiledQueryExample(employeeCache);
 
                 // Run SQL query with join example.
-                JoinQueryExample(employeeCache, organizationCache);
+                JoinQueryExample(employeeCacheColocated, organizationCache);
+
+                // Run SQL query with distributed join example.
+                DistributedJoinQueryExample(employeeCache, organizationCache);
 
                 // Run SQL fields query example.
                 FieldsQueryExample(employeeCache);
@@ -103,17 +101,17 @@ namespace Apache.Ignite.Examples.Datagrid
         /// Queries employees that have provided ZIP code in address.
         /// </summary>
         /// <param name="cache">Cache.</param>
-        private static void QueryExample(ICache<EmployeeKey, Employee> cache)
+        private static void QueryExample(ICache<int, Employee> cache)
         {
             const int zip = 94109;
 
-            IQueryable<ICacheEntry<EmployeeKey, Employee>> qry =
+            IQueryable<ICacheEntry<int, Employee>> qry =
                 cache.AsCacheQueryable().Where(emp => emp.Value.Address.Zip == zip);
 
             Console.WriteLine();
             Console.WriteLine(">>> Employees with zipcode " + zip + ":");
 
-            foreach (ICacheEntry<EmployeeKey, Employee> entry in qry)
+            foreach (ICacheEntry<int, Employee> entry in qry)
                 Console.WriteLine(">>>    " + entry.Value);
         }
 
@@ -121,18 +119,18 @@ namespace Apache.Ignite.Examples.Datagrid
         /// Queries employees that have provided ZIP code in address with a compiled query.
         /// </summary>
         /// <param name="cache">Cache.</param>
-        private static void CompiledQueryExample(ICache<EmployeeKey, Employee> cache)
+        private static void CompiledQueryExample(ICache<int, Employee> cache)
         {
             const int zip = 94109;
 
             // Compile cache query to eliminate LINQ overhead on multiple runs.
-            Func<int, IQueryCursor<ICacheEntry<EmployeeKey, Employee>>> qry =
+            Func<int, IQueryCursor<ICacheEntry<int, Employee>>> qry =
                 CompiledQuery.Compile((int z) => cache.AsCacheQueryable().Where(emp => emp.Value.Address.Zip == z));
 
             Console.WriteLine();
             Console.WriteLine(">>> Employees with zipcode using compiled query " + zip + ":");
 
-            foreach (ICacheEntry<EmployeeKey, Employee> entry in qry(zip))
+            foreach (ICacheEntry<int, Employee> entry in qry(zip))
                 Console.WriteLine(">>>    " + entry.Value);
         }
 
@@ -141,25 +139,54 @@ namespace Apache.Ignite.Examples.Datagrid
         /// </summary>
         /// <param name="employeeCache">Employee cache.</param>
         /// <param name="organizationCache">Organization cache.</param>
-        private static void JoinQueryExample(ICache<EmployeeKey, Employee> employeeCache,
+        private static void JoinQueryExample(ICache<AffinityKey, Employee> employeeCache,
             ICache<int, Organization> organizationCache)
         {
             const string orgName = "Apache";
 
-            IQueryable<ICacheEntry<EmployeeKey, Employee>> employees = employeeCache.AsCacheQueryable();
+            IQueryable<ICacheEntry<AffinityKey, Employee>> employees = employeeCache.AsCacheQueryable();
             IQueryable<ICacheEntry<int, Organization>> organizations = organizationCache.AsCacheQueryable();
 
-            IQueryable<ICacheEntry<EmployeeKey, Employee>> qry =
+            IQueryable<ICacheEntry<AffinityKey, Employee>> qry =
+                from employee in employees
+                from organization in organizations
+                where employee.Value.OrganizationId == organization.Key && organization.Value.Name == orgName
+                select employee;
+
+
+            Console.WriteLine();
+            Console.WriteLine(">>> Employees working for " + orgName + ":");
+
+            foreach (ICacheEntry<AffinityKey, Employee> entry in qry)
+                Console.WriteLine(">>>     " + entry.Value);
+        }
+
+        /// <summary>
+        /// Queries employees that work for organization with provided name.
+        /// </summary>
+        /// <param name="employeeCache">Employee cache.</param>
+        /// <param name="organizationCache">Organization cache.</param>
+        private static void DistributedJoinQueryExample(ICache<int, Employee> employeeCache,
+            ICache<int, Organization> organizationCache)
+        {
+            const string orgName = "Apache";
+
+            var queryOptions = new QueryOptions {EnableDistributedJoins = true};
+
+            IQueryable<ICacheEntry<int, Employee>> employees = employeeCache.AsCacheQueryable(queryOptions);
+            IQueryable<ICacheEntry<int, Organization>> organizations = organizationCache.AsCacheQueryable(queryOptions);
+
+            IQueryable<ICacheEntry<int, Employee>> qry =
                 from employee in employees
                 from organization in organizations
-                where employee.Key.OrganizationId == organization.Key && organization.Value.Name == orgName
+                where employee.Value.OrganizationId == organization.Key && organization.Value.Name == orgName
                 select employee;
 
 
             Console.WriteLine();
             Console.WriteLine(">>> Employees working for " + orgName + ":");
 
-            foreach (ICacheEntry<EmployeeKey, Employee> entry in qry)
+            foreach (ICacheEntry<int, Employee> entry in qry)
                 Console.WriteLine(">>>     " + entry.Value);
         }
 
@@ -167,7 +194,7 @@ namespace Apache.Ignite.Examples.Datagrid
         /// Queries names and salaries for all employees.
         /// </summary>
         /// <param name="cache">Cache.</param>
-        private static void FieldsQueryExample(ICache<EmployeeKey, Employee> cache)
+        private static void FieldsQueryExample(ICache<int, Employee> cache)
         {
             var qry = cache.AsCacheQueryable().Select(entry => new {entry.Value.Name, entry.Value.Salary});
 
@@ -188,71 +215,125 @@ namespace Apache.Ignite.Examples.Datagrid
                 "Apache",
                 new Address("1065 East Hillsdale Blvd, Foster City, CA", 94404),
                 OrganizationType.Private,
-                DateTime.Now
-            ));
+                DateTime.Now));
 
             cache.Put(2, new Organization(
                 "Microsoft",
                 new Address("1096 Eddy Street, San Francisco, CA", 94109),
                 OrganizationType.Private,
-                DateTime.Now
-            ));
+                DateTime.Now));
+        }
+
+        /// <summary>
+        /// Populate cache with data for this example.
+        /// </summary>
+        /// <param name="cache">Cache.</param>
+        private static void PopulateCache(ICache<AffinityKey, Employee> cache)
+        {
+            cache.Put(new AffinityKey(1, 1), new Employee(
+                "James Wilson",
+                12500,
+                new Address("1096 Eddy Street, San Francisco, CA", 94109),
+                new[] {"Human Resources", "Customer Service"},
+                1));
+
+            cache.Put(new AffinityKey(2, 1), new Employee(
+                "Daniel Adams",
+                11000,
+                new Address("184 Fidler Drive, San Antonio, TX", 78130),
+                new[] {"Development", "QA"},
+                1));
+
+            cache.Put(new AffinityKey(3, 1), new Employee(
+                "Cristian Moss",
+                12500,
+                new Address("667 Jerry Dove Drive, Florence, SC", 29501),
+                new[] {"Logistics"},
+                1));
+
+            cache.Put(new AffinityKey(4, 2), new Employee(
+                "Allison Mathis",
+                25300,
+                new Address("2702 Freedom Lane, San Francisco, CA", 94109),
+                new[] {"Development"},
+                2));
+
+            cache.Put(new AffinityKey(5, 2), new Employee(
+                "Breana Robbin",
+                6500,
+                new Address("3960 Sundown Lane, Austin, TX", 78130),
+                new[] {"Sales"},
+                2));
+
+            cache.Put(new AffinityKey(6, 2), new Employee(
+                "Philip Horsley",
+                19800,
+                new Address("2803 Elsie Drive, Sioux Falls, SD", 57104),
+                new[] {"Sales"},
+                2));
+
+            cache.Put(new AffinityKey(7, 2), new Employee(
+                "Brian Peters",
+                10600,
+                new Address("1407 Pearlman Avenue, Boston, MA", 12110),
+                new[] {"Development", "QA"},
+                2));
         }
 
         /// <summary>
         /// Populate cache with data for this example.
         /// </summary>
         /// <param name="cache">Cache.</param>
-        private static void PopulateCache(ICache<EmployeeKey, Employee> cache)
+        private static void PopulateCache(ICache<int, Employee> cache)
         {
-            cache.Put(new EmployeeKey(1, 1), new Employee(
+            cache.Put(1, new Employee(
                 "James Wilson",
                 12500,
                 new Address("1096 Eddy Street, San Francisco, CA", 94109),
-                new List<string> { "Human Resources", "Customer Service" }
-            ));
+                new[] {"Human Resources", "Customer Service"},
+                1));
 
-            cache.Put(new EmployeeKey(2, 1), new Employee(
+            cache.Put(2, new Employee(
                 "Daniel Adams",
                 11000,
                 new Address("184 Fidler Drive, San Antonio, TX", 78130),
-                new List<string> { "Development", "QA" }
-            ));
+                new[] {"Development", "QA"},
+                1));
 
-            cache.Put(new EmployeeKey(3, 1), new Employee(
+            cache.Put(3, new Employee(
                 "Cristian Moss",
                 12500,
                 new Address("667 Jerry Dove Drive, Florence, SC", 29501),
-                new List<string> { "Logistics" }
-            ));
+                new[] {"Logistics"},
+                1));
 
-            cache.Put(new EmployeeKey(4, 2), new Employee(
+            cache.Put(4, new Employee(
                 "Allison Mathis",
                 25300,
                 new Address("2702 Freedom Lane, San Francisco, CA", 94109),
-                new List<string> { "Development" }
-            ));
+                new[] {"Development"},
+                2));
 
-            cache.Put(new EmployeeKey(5, 2), new Employee(
+            cache.Put(5, new Employee(
                 "Breana Robbin",
                 6500,
                 new Address("3960 Sundown Lane, Austin, TX", 78130),
-                new List<string> { "Sales" }
-            ));
+                new[] {"Sales"},
+                2));
 
-            cache.Put(new EmployeeKey(6, 2), new Employee(
+            cache.Put(6, new Employee(
                 "Philip Horsley",
                 19800,
                 new Address("2803 Elsie Drive, Sioux Falls, SD", 57104),
-                new List<string> { "Sales" }
-            ));
+                new[] {"Sales"},
+                2));
 
-            cache.Put(new EmployeeKey(7, 2), new Employee(
+            cache.Put(7, new Employee(
                 "Brian Peters",
                 10600,
                 new Address("1407 Pearlman Avenue, Boston, MA", 12110),
-                new List<string> { "Development", "QA" }
-            ));
+                new[] {"Development", "QA"},
+                2));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/861c1736/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs
index 8b5e6f3..1c35149 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs
@@ -22,6 +22,7 @@ namespace Apache.Ignite.Examples.Datagrid
     using System.Collections.Generic;
     using Apache.Ignite.Core;
     using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Affinity;
     using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Cache.Query;
     using Apache.Ignite.ExamplesDll.Binary;
@@ -49,6 +50,9 @@ namespace Apache.Ignite.Examples.Datagrid
         /// <summary>Employee cache name.</summary>
         private const string EmployeeCacheName = "dotnet_cache_query_employee";
 
+        /// <summary>Employee cache name.</summary>
+        private const string EmployeeCacheNameColocated = "dotnet_cache_query_employee_colocated";
+
         [STAThread]
         public static void Main()
         {
@@ -57,33 +61,28 @@ namespace Apache.Ignite.Examples.Datagrid
                 Console.WriteLine();
                 Console.WriteLine(">>> Cache query example started.");
 
-                var employeeCache = ignite.GetOrCreateCache<EmployeeKey, Employee>(new CacheConfiguration
-                {
-                    Name = EmployeeCacheName,
-                    QueryEntities = new[]
-                    {
-                        new QueryEntity(typeof(EmployeeKey), typeof(Employee))
-                    }
-                });
-
-                var organizationCache = ignite.GetOrCreateCache<int, Organization>(new CacheConfiguration
-                {
-                    Name = OrganizationCacheName,
-                    QueryEntities = new[]
-                    {
-                        new QueryEntity(typeof(int), typeof(Organization))
-                    }
-                });
+                var employeeCache = ignite.GetOrCreateCache<int, Employee>(
+                    new CacheConfiguration(EmployeeCacheName, typeof(Employee)));
+
+                var employeeCacheColocated = ignite.GetOrCreateCache<AffinityKey, Employee>(
+                    new CacheConfiguration(EmployeeCacheNameColocated, typeof(Employee)));
+
+                var organizationCache = ignite.GetOrCreateCache<int, Organization>(
+                    new CacheConfiguration(OrganizationCacheName, new QueryEntity(typeof(int), typeof(Organization))));
 
                 // Populate cache with sample data entries.
                 PopulateCache(employeeCache);
+                PopulateCache(employeeCacheColocated);
                 PopulateCache(organizationCache);
 
                 // Run SQL query example.
                 SqlQueryExample(employeeCache);
 
                 // Run SQL query with join example.
-                SqlJoinQueryExample(employeeCache);
+                SqlJoinQueryExample(employeeCacheColocated);
+
+                // Run SQL query with distributed join example.
+                SqlDistributedJoinQueryExample(employeeCache);
 
                 // Run SQL fields query example.
                 SqlFieldsQueryExample(employeeCache);
@@ -103,7 +102,7 @@ namespace Apache.Ignite.Examples.Datagrid
         /// Queries employees that have provided ZIP code in address.
         /// </summary>
         /// <param name="cache">Cache.</param>
-        private static void SqlQueryExample(ICache<EmployeeKey, Employee> cache)
+        private static void SqlQueryExample(ICache<int, Employee> cache)
         {
             const int zip = 94109;
 
@@ -120,7 +119,7 @@ namespace Apache.Ignite.Examples.Datagrid
         /// Queries employees that work for organization with provided name.
         /// </summary>
         /// <param name="cache">Cache.</param>
-        private static void SqlJoinQueryExample(ICache<EmployeeKey, Employee> cache)
+        private static void SqlJoinQueryExample(ICache<AffinityKey, Employee> cache)
         {
             const string orgName = "Apache";
 
@@ -136,10 +135,32 @@ namespace Apache.Ignite.Examples.Datagrid
         }
 
         /// <summary>
+        /// Queries employees that work for organization with provided name.
+        /// </summary>
+        /// <param name="cache">Cache.</param>
+        private static void SqlDistributedJoinQueryExample(ICache<int, Employee> cache)
+        {
+            const string orgName = "Apache";
+
+            var qry = cache.Query(new SqlQuery("Employee",
+                "from Employee, \"dotnet_cache_query_organization\".Organization " +
+                "where Employee.organizationId = Organization._key and Organization.name = ?", orgName)
+            {
+                EnableDistributedJoins = true
+            });
+
+            Console.WriteLine();
+            Console.WriteLine(">>> Employees working for " + orgName + ":");
+
+            foreach (var entry in qry)
+                Console.WriteLine(">>>     " + entry.Value);
+        }
+
+        /// <summary>
         /// Queries names and salaries for all employees.
         /// </summary>
         /// <param name="cache">Cache.</param>
-        private static void SqlFieldsQueryExample(ICache<EmployeeKey, Employee> cache)
+        private static void SqlFieldsQueryExample(ICache<int, Employee> cache)
         {
             var qry = cache.QueryFields(new SqlFieldsQuery("select name, salary from Employee"));
 
@@ -154,7 +175,7 @@ namespace Apache.Ignite.Examples.Datagrid
         /// Queries employees that live in Texas using full-text query API.
         /// </summary>
         /// <param name="cache">Cache.</param>
-        private static void FullTextQueryExample(ICache<EmployeeKey, Employee> cache)
+        private static void FullTextQueryExample(ICache<int, Employee> cache)
         {
             var qry = cache.Query(new TextQuery("Employee", "TX"));
 
@@ -175,71 +196,124 @@ namespace Apache.Ignite.Examples.Datagrid
                 "Apache",
                 new Address("1065 East Hillsdale Blvd, Foster City, CA", 94404),
                 OrganizationType.Private,
-                DateTime.Now
-            ));
+                DateTime.Now));
 
-            cache.Put(2, new Organization(
-                "Microsoft",
+            cache.Put(2, new Organization("Microsoft",
                 new Address("1096 Eddy Street, San Francisco, CA", 94109),
                 OrganizationType.Private,
-                DateTime.Now
-            ));
+                DateTime.Now));
+        }
+
+        /// <summary>
+        /// Populate cache with data for this example.
+        /// </summary>
+        /// <param name="cache">Cache.</param>
+        private static void PopulateCache(ICache<AffinityKey, Employee> cache)
+        {
+            cache.Put(new AffinityKey(1, 1), new Employee(
+                "James Wilson",
+                12500,
+                new Address("1096 Eddy Street, San Francisco, CA", 94109),
+                new[] {"Human Resources", "Customer Service"},
+                1));
+
+            cache.Put(new AffinityKey(2, 1), new Employee(
+                "Daniel Adams",
+                11000,
+                new Address("184 Fidler Drive, San Antonio, TX", 78130),
+                new[] {"Development", "QA"},
+                1));
+
+            cache.Put(new AffinityKey(3, 1), new Employee(
+                "Cristian Moss",
+                12500,
+                new Address("667 Jerry Dove Drive, Florence, SC", 29501),
+                new[] {"Logistics"},
+                1));
+
+            cache.Put(new AffinityKey(4, 2), new Employee(
+                "Allison Mathis",
+                25300,
+                new Address("2702 Freedom Lane, San Francisco, CA", 94109),
+                new[] {"Development"},
+                2));
+
+            cache.Put(new AffinityKey(5, 2), new Employee(
+                "Breana Robbin",
+                6500,
+                new Address("3960 Sundown Lane, Austin, TX", 78130),
+                new[] {"Sales"},
+                2));
+
+            cache.Put(new AffinityKey(6, 2), new Employee(
+                "Philip Horsley",
+                19800,
+                new Address("2803 Elsie Drive, Sioux Falls, SD", 57104),
+                new[] {"Sales"},
+                2));
+
+            cache.Put(new AffinityKey(7, 2), new Employee(
+                "Brian Peters",
+                10600,
+                new Address("1407 Pearlman Avenue, Boston, MA", 12110),
+                new[] {"Development", "QA"},
+                2));
         }
 
         /// <summary>
         /// Populate cache with data for this example.
         /// </summary>
         /// <param name="cache">Cache.</param>
-        private static void PopulateCache(ICache<EmployeeKey, Employee> cache)
+        private static void PopulateCache(ICache<int, Employee> cache)
         {
-            cache.Put(new EmployeeKey(1, 1), new Employee(
+            cache.Put(1, new Employee(
                 "James Wilson",
                 12500,
                 new Address("1096 Eddy Street, San Francisco, CA", 94109),
-                new List<string> { "Human Resources", "Customer Service" }
-            ));
+                new[] {"Human Resources", "Customer Service"},
+                1));
 
-            cache.Put(new EmployeeKey(2, 1), new Employee(
+            cache.Put(2, new Employee(
                 "Daniel Adams",
                 11000,
                 new Address("184 Fidler Drive, San Antonio, TX", 78130),
-                new List<string> { "Development", "QA" }
-            ));
+                new[] {"Development", "QA"},
+                1));
 
-            cache.Put(new EmployeeKey(3, 1), new Employee(
+            cache.Put(3, new Employee(
                 "Cristian Moss",
                 12500,
                 new Address("667 Jerry Dove Drive, Florence, SC", 29501),
-                new List<string> { "Logistics" }
-            ));
+                new[] {"Logistics"},
+                1));
 
-            cache.Put(new EmployeeKey(4, 2), new Employee(
+            cache.Put(4, new Employee(
                 "Allison Mathis",
                 25300,
                 new Address("2702 Freedom Lane, San Francisco, CA", 94109),
-                new List<string> { "Development" }
-            ));
+                new[] {"Development"},
+                2));
 
-            cache.Put(new EmployeeKey(5, 2), new Employee(
+            cache.Put(5, new Employee(
                 "Breana Robbin",
                 6500,
                 new Address("3960 Sundown Lane, Austin, TX", 78130),
-                new List<string> { "Sales" }
-            ));
+                new[] {"Sales"},
+                2));
 
-            cache.Put(new EmployeeKey(6, 2), new Employee(
+            cache.Put(6, new Employee(
                 "Philip Horsley",
                 19800,
                 new Address("2803 Elsie Drive, Sioux Falls, SD", 57104),
-                new List<string> { "Sales" }
-            ));
+                new[] {"Sales"},
+                2));
 
-            cache.Put(new EmployeeKey(7, 2), new Employee(
+            cache.Put(7, new Employee(
                 "Brian Peters",
                 10600,
                 new Address("1407 Pearlman Avenue, Boston, MA", 12110),
-                new List<string> { "Development", "QA" }
-            ));
+                new[] {"Development", "QA"},
+                2));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/861c1736/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj
index 16d0be3..41981d8 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj
@@ -57,7 +57,6 @@
     <Compile Include="Binary\Account.cs" />
     <Compile Include="Binary\Address.cs" />
     <Compile Include="Binary\Employee.cs" />
-    <Compile Include="Binary\EmployeeKey.cs" />
     <Compile Include="Binary\Organization.cs" />
     <Compile Include="Binary\OrganizationType.cs" />
     <Compile Include="Properties\AssemblyInfo.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/861c1736/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs
index 0fc3230..4cff2a8 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs
@@ -35,12 +35,14 @@ namespace Apache.Ignite.ExamplesDll.Binary
         /// <param name="salary">Salary.</param>
         /// <param name="address">Address.</param>
         /// <param name="departments">Departments.</param>
-        public Employee(string name, long salary, Address address, ICollection<string> departments)
+        public Employee(string name, long salary, Address address, ICollection<string> departments, 
+            int organizationId = 0)
         {
             Name = name;
             Salary = salary;
             Address = address;
             Departments = departments;
+            OrganizationId = organizationId;
         }
 
         /// <summary>
@@ -50,6 +52,12 @@ namespace Apache.Ignite.ExamplesDll.Binary
         public string Name { get; set; }
 
         /// <summary>
+        /// Organization id.
+        /// </summary>
+        [QuerySqlField(IsIndexed = true)]
+        public int OrganizationId { get; set; }
+
+        /// <summary>
         /// Salary.
         /// </summary>
         [QuerySqlField]

http://git-wip-us.apache.org/repos/asf/ignite/blob/861c1736/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/EmployeeKey.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/EmployeeKey.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/EmployeeKey.cs
deleted file mode 100644
index e699648..0000000
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/EmployeeKey.cs
+++ /dev/null
@@ -1,88 +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.
- */
-
-namespace Apache.Ignite.ExamplesDll.Binary
-{
-    using Apache.Ignite.Core.Cache.Affinity;
-    using Apache.Ignite.Core.Cache.Configuration;
-
-    /// <summary>
-    /// Employee key. Used in query example to co-locate employees with their organizations.
-    /// </summary>
-    public class EmployeeKey
-    {
-        /// <summary>
-        /// Constructor.
-        /// </summary>
-        /// <param name="id">ID.</param>
-        /// <param name="orgId">Organization ID.</param>
-        public EmployeeKey(int id, int orgId)
-        {
-            Id = id;
-            OrganizationId = orgId;
-        }
-
-        /// <summary>
-        /// ID.
-        /// </summary>
-        public int Id { get; private set; }
-
-        /// <summary>
-        /// Organization ID.
-        /// </summary>
-        [AffinityKeyMapped]
-        [QuerySqlField(IsIndexed = true)]
-        public int OrganizationId { get; private set; }
-
-        /// <summary>
-        /// Determines whether the specified <see cref="T:System.Object"/> is equal to the current <see cref="T:System.Object"/>.
-        /// </summary>
-        /// <returns>
-        /// true if the specified <see cref="T:System.Object"/> is equal to the current <see cref="T:System.Object"/>; otherwise, false.
-        /// </returns>
-        /// <param name="obj">The object to compare with the current object. </param><filterpriority>2</filterpriority>
-        public override bool Equals(object obj)
-        {
-            EmployeeKey other = obj as EmployeeKey;
-
-            return other != null && Id == other.Id && OrganizationId == other.OrganizationId;
-        }
-
-        /// <summary>
-        /// Serves as a hash function for a particular type. 
-        /// </summary>
-        /// <returns>
-        /// A hash code for the current <see cref="T:System.Object"/>.
-        /// </returns>
-        /// <filterpriority>2</filterpriority>
-        public override int GetHashCode()
-        {
-            return 31 * Id + OrganizationId;
-        }
-
-        /// <summary>
-        /// Returns a string that represents the current object.
-        /// </summary>
-        /// <returns>
-        /// A string that represents the current object.
-        /// </returns>
-        public override string ToString()
-        {
-            return string.Format("{0} [id={1}, organizationId={2}]", typeof(EmployeeKey).Name, Id, OrganizationId);
-        }
-    }
-}


[14/50] [abbrv] ignite git commit: Restored services compatibility.

Posted by vo...@apache.org.
Restored services compatibility.


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

Branch: refs/heads/ignite-2693
Commit: 92fff630fbf36c82f93bbd9ddd53d11bed44e772
Parents: 61ab650
Author: devozerov <vo...@gridgain.com>
Authored: Wed Nov 2 17:50:51 2016 +0300
Committer: thatcoach <pp...@list.ru>
Committed: Wed Nov 2 17:51:06 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/job/GridJobWorker.java  |  10 +-
 .../service/GridServiceProcessor.java           | 197 +++++++++++--------
 .../internal/util/SerializableTransient.java    |  58 ++++++
 .../ignite/marshaller/MarshallerUtils.java      |  22 +++
 .../optimized/OptimizedClassDescriptor.java     |  90 ++++++++-
 5 files changed, 296 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/92fff630/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java
index 8169eb1..5f38b29 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java
@@ -57,6 +57,7 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteRunnable;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.marshaller.MarshallerUtils;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.events.EventType.EVT_JOB_CANCELLED;
@@ -421,7 +422,14 @@ public class GridJobWorker extends GridWorker implements GridTimeoutObject {
 
         try {
             if (job == null) {
-                job = U.unmarshal(marsh, jobBytes, U.resolveClassLoader(dep.classLoader(), ctx.config()));
+                MarshallerUtils.jobSenderVersion(taskNode.version());
+
+                try {
+                    job = U.unmarshal(marsh, jobBytes, U.resolveClassLoader(dep.classLoader(), ctx.config()));
+                }
+                finally {
+                    MarshallerUtils.jobSenderVersion(null);
+                }
 
                 // No need to hold reference any more.
                 jobBytes = null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/92fff630/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 527d360..8489875 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -20,11 +20,14 @@ package org.apache.ignite.internal.processors.service;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentMap;
@@ -87,6 +90,7 @@ import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.internal.util.SerializableTransient;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.resources.JobContextResource;
 import org.apache.ignite.resources.LoggerResource;
@@ -115,6 +119,9 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     /** */
     public static final IgniteProductVersion LAZY_SERVICES_CFG_SINCE = IgniteProductVersion.fromString("1.5.22");
 
+    /** Versions that only compatible with each other, and from 1.5.33. */
+    private static final Set<IgniteProductVersion> SERVICE_TOP_CALLABLE_VER1;
+
     /** */
     private final Boolean srvcCompatibilitySysProp;
 
@@ -162,6 +169,31 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     /** Topology listener. */
     private GridLocalEventListener topLsnr = new TopologyListener();
 
+    static {
+        Set<IgniteProductVersion> versions = new TreeSet<>(new Comparator<IgniteProductVersion>() {
+            @Override public int compare(final IgniteProductVersion o1, final IgniteProductVersion o2) {
+                return o1.compareToIgnoreTimestamp(o2);
+            }
+        });
+
+        versions.add(IgniteProductVersion.fromString("1.5.30"));
+        versions.add(IgniteProductVersion.fromString("1.5.31"));
+        versions.add(IgniteProductVersion.fromString("1.5.32"));
+        versions.add(IgniteProductVersion.fromString("1.6.3"));
+        versions.add(IgniteProductVersion.fromString("1.6.4"));
+        versions.add(IgniteProductVersion.fromString("1.6.5"));
+        versions.add(IgniteProductVersion.fromString("1.6.6"));
+        versions.add(IgniteProductVersion.fromString("1.6.7"));
+        versions.add(IgniteProductVersion.fromString("1.6.8"));
+        versions.add(IgniteProductVersion.fromString("1.6.9"));
+        versions.add(IgniteProductVersion.fromString("1.6.10"));
+        versions.add(IgniteProductVersion.fromString("1.7.0"));
+        versions.add(IgniteProductVersion.fromString("1.7.1"));
+        versions.add(IgniteProductVersion.fromString("1.7.2"));
+
+        SERVICE_TOP_CALLABLE_VER1 = Collections.unmodifiableSet(versions);
+    }
+
     /**
      * @param ctx Kernal context.
      */
@@ -668,9 +700,13 @@ public class GridServiceProcessor extends GridProcessorAdapter {
         ClusterNode node = cache.affinity().mapKeyToNode(name);
 
         if (node.version().compareTo(ServiceTopologyCallable.SINCE_VER) >= 0) {
+            final ServiceTopologyCallable call = new ServiceTopologyCallable(name);
+
+            call.serialize = SERVICE_TOP_CALLABLE_VER1.contains(node.version());
+
             return ctx.closure().callAsyncNoFailover(
                 GridClosureCallMode.BROADCAST,
-                new ServiceTopologyCallable(name),
+                call,
                 Collections.singletonList(node),
                 false
             ).get();
@@ -815,7 +851,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
             }
         }
 
-        return new GridServiceProxy<T>(prj, name, svcItf, sticky, ctx).proxy();
+        return new GridServiceProxy<>(prj, name, svcItf, sticky, ctx).proxy();
     }
 
     /**
@@ -868,7 +904,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
      * @param topVer Topology version.
      * @throws IgniteCheckedException If failed.
      */
-    private void reassign(GridServiceDeployment dep, AffinityTopologyVersion topVer) throws IgniteCheckedException {
+    private void reassign(GridServiceDeployment dep, long topVer) throws IgniteCheckedException {
         ServiceConfiguration cfg = dep.configuration();
 
         Object nodeFilter = cfg.getNodeFilter();
@@ -882,7 +918,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
         Object affKey = cfg.getAffinityKey();
 
         while (true) {
-            GridServiceAssignments assigns = new GridServiceAssignments(cfg, dep.nodeId(), topVer.topologyVersion());
+            GridServiceAssignments assigns = new GridServiceAssignments(cfg, dep.nodeId(), topVer);
 
              Collection<ClusterNode> nodes;
 
@@ -912,7 +948,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                 Map<UUID, Integer> cnts = new HashMap<>();
 
                 if (affKey != null) {
-                    ClusterNode n = ctx.affinity().mapKeyToNode(cacheName, affKey, topVer);
+                    ClusterNode n = ctx.affinity().mapKeyToNode(cacheName, affKey, new AffinityTopologyVersion(topVer));
 
                     if (n != null) {
                         int cnt = maxPerNodeCnt == 0 ? totalCnt == 0 ? 1 : totalCnt : maxPerNodeCnt;
@@ -1144,7 +1180,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
         if (cfg instanceof LazyServiceConfiguration) {
             byte[] bytes = ((LazyServiceConfiguration)cfg).serviceBytes();
 
-            Service srvc = U.unmarshal(m, bytes, U.resolveClassLoader(null, ctx.config()));
+            Service srvc = m.unmarshal(bytes, U.resolveClassLoader(null, ctx.config()));
 
             ctx.resource().inject(srvc);
 
@@ -1154,9 +1190,10 @@ public class GridServiceProcessor extends GridProcessorAdapter {
             Service svc = cfg.getService();
 
             try {
-                byte[] bytes = U.marshal(m, svc);
+                byte[] bytes = m.marshal(svc);
 
-                Service cp = U.unmarshal(m, bytes, U.resolveClassLoader(svc.getClass().getClassLoader(), ctx.config()));
+                Service cp = m.unmarshal(bytes,
+                    U.resolveClassLoader(svc.getClass().getClassLoader(), ctx.config()));
 
                 ctx.resource().inject(cp);
 
@@ -1231,8 +1268,8 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                 ClusterNode oldestSrvNode =
                     CU.oldestAliveCacheServerNode(cache.context().shared(), AffinityTopologyVersion.NONE);
 
-                if (oldestSrvNode == null)
-                    return new GridEmptyIterator<>();
+            if (oldestSrvNode == null)
+                return F.emptyIterator();
 
                 GridCacheQueryManager qryMgr = cache.context().queries();
 
@@ -1418,7 +1455,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
             svcName.set(dep.configuration().getName());
 
             // Ignore other utility cache events.
-            AffinityTopologyVersion topVer = ctx.discovery().topologyVersionEx();
+            long topVer = ctx.discovery().topologyVersion();
 
             ClusterNode oldest = U.oldest(ctx.discovery().nodes(topVer), null);
 
@@ -1469,60 +1506,60 @@ public class GridServiceProcessor extends GridProcessorAdapter {
         }
     }
 
-        /**
-         * Deployment callback.
-         *
-         * @param dep Service deployment.
-         * @param topVer Topology version.
-         */
-        private void onDeployment(final GridServiceDeployment dep, final AffinityTopologyVersion topVer) {
-            // Retry forever.
-            try {
-                AffinityTopologyVersion newTopVer = ctx.discovery().topologyVersionEx();
+    /**
+     * Deployment callback.
+     *
+     * @param dep Service deployment.
+     * @param topVer Topology version.
+     */
+    private void onDeployment(final GridServiceDeployment dep, final long topVer) {
+        // Retry forever.
+        try {
+            long newTopVer = ctx.discovery().topologyVersion();
 
-                // If topology version changed, reassignment will happen from topology event.
-                if (newTopVer.equals(topVer))
-                    reassign(dep, topVer);
-            }
-            catch (IgniteCheckedException e) {
-                if (!(e instanceof ClusterTopologyCheckedException))
-                    log.error("Failed to do service reassignment (will retry): " + dep.configuration().getName(), e);
+            // If topology version changed, reassignment will happen from topology event.
+            if (newTopVer == topVer)
+                reassign(dep, topVer);
+        }
+        catch (IgniteCheckedException e) {
+            if (!(e instanceof ClusterTopologyCheckedException))
+                log.error("Failed to do service reassignment (will retry): " + dep.configuration().getName(), e);
 
-                AffinityTopologyVersion newTopVer = ctx.discovery().topologyVersionEx();
+            long newTopVer = ctx.discovery().topologyVersion();
 
-                if (!newTopVer.equals(topVer)) {
-                    assert newTopVer.compareTo(topVer) > 0;
+            if (newTopVer != topVer) {
+                assert newTopVer > topVer;
 
-                    // Reassignment will happen from topology event.
-                    return;
-                }
+                // Reassignment will happen from topology event.
+                return;
+            }
 
-                ctx.timeout().addTimeoutObject(new GridTimeoutObject() {
-                    private IgniteUuid id = IgniteUuid.randomUuid();
+            ctx.timeout().addTimeoutObject(new GridTimeoutObject() {
+                private IgniteUuid id = IgniteUuid.randomUuid();
 
-                    private long start = System.currentTimeMillis();
+                private long start = System.currentTimeMillis();
 
-                    @Override public IgniteUuid timeoutId() {
-                        return id;
-                    }
+                @Override public IgniteUuid timeoutId() {
+                    return id;
+                }
 
-                    @Override public long endTime() {
-                        return start + RETRY_TIMEOUT;
-                    }
+                @Override public long endTime() {
+                    return start + RETRY_TIMEOUT;
+                }
 
-                    @Override public void onTimeout() {
-                        if (!busyLock.enterBusy())
-                            return;
+                @Override public void onTimeout() {
+                    if (!busyLock.enterBusy())
+                        return;
 
-                        try {
-                            // Try again.
-                            onDeployment(dep, topVer);
-                        }
-                        finally {
-                            busyLock.leaveBusy();
-                        }
+                    try {
+                        // Try again.
+                        onDeployment(dep, topVer);
                     }
-                });
+                    finally {
+                        busyLock.leaveBusy();
+                    }
+                }
+            });
         }
     }
 
@@ -1531,28 +1568,16 @@ public class GridServiceProcessor extends GridProcessorAdapter {
      */
     private class TopologyListener implements GridLocalEventListener {
         /** {@inheritDoc} */
-        @Override public void onEvent(Event evt) {
+        @Override public void onEvent(final Event evt) {
             if (!busyLock.enterBusy())
                 return;
 
             try {
-                final AffinityTopologyVersion topVer;
-
-                if (evt instanceof DiscoveryCustomEvent) {
-                    DiscoveryCustomMessage msg = ((DiscoveryCustomEvent)evt).customMessage();
-
-                    topVer = ((DiscoveryCustomEvent)evt).affinityTopologyVersion();
-
-                    if (msg instanceof CacheAffinityChangeMessage) {
-                        if (!((CacheAffinityChangeMessage)msg).exchangeNeeded())
-                            return;
-                    }
-                }
-                else
-                    topVer = new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion(), 0);
-
                 depExe.submit(new BusyRunnable() {
                     @Override public void run0() {
+                        AffinityTopologyVersion topVer =
+                            new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion());
+
                         ClusterNode oldest = CU.oldestAliveCacheServerNode(cache.context().shared(), topVer);
 
                         if (oldest != null && oldest.isLocal()) {
@@ -1587,7 +1612,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                                         ctx.cache().internalCache(UTILITY_CACHE_NAME).context().affinity().
                                             affinityReadyFuture(topVer).get();
 
-                                        reassign(dep, topVer);
+                                        reassign(dep, topVer.topologyVersion());
                                     }
                                     catch (IgniteCheckedException ex) {
                                         if (!(e instanceof ClusterTopologyCheckedException))
@@ -1604,7 +1629,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                             }
 
                             if (!retries.isEmpty())
-                                onReassignmentFailed(topVer, retries);
+                                onReassignmentFailed(topVer.topologyVersion(), retries);
                         }
 
                         // Clean up zombie assignments.
@@ -1641,14 +1666,13 @@ public class GridServiceProcessor extends GridProcessorAdapter {
          * @param topVer Topology version.
          * @param retries Retries.
          */
-        private void onReassignmentFailed(final AffinityTopologyVersion topVer,
-            final Collection<GridServiceDeployment> retries) {
+        private void onReassignmentFailed(final long topVer, final Collection<GridServiceDeployment> retries) {
             if (!busyLock.enterBusy())
                 return;
 
             try {
                 // If topology changed again, let next event handle it.
-                if (ctx.discovery().topologyVersionEx().equals(topVer))
+                if (ctx.discovery().topologyVersion() != topVer)
                     return;
 
                 for (Iterator<GridServiceDeployment> it = retries.iterator(); it.hasNext(); ) {
@@ -1829,6 +1853,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     /**
      */
     @GridInternal
+    @SerializableTransient(methodName = "serializableTransient")
     private static class ServiceTopologyCallable implements IgniteCallable<Map<UUID, Integer>> {
         /** */
         private static final long serialVersionUID = 0L;
@@ -1837,10 +1862,13 @@ public class GridServiceProcessor extends GridProcessorAdapter {
         private static final IgniteProductVersion SINCE_VER = IgniteProductVersion.fromString("1.5.7");
 
         /** */
+        private static final String[] SER_FIELDS = {"waitedCacheInit", "jCtx", "log"};
+
+        /** */
         private final String svcName;
 
         /** */
-        private boolean waitedCacheInit;
+        private transient boolean waitedCacheInit;
 
         /** */
         @IgniteInstanceResource
@@ -1848,11 +1876,14 @@ public class GridServiceProcessor extends GridProcessorAdapter {
 
         /** */
         @JobContextResource
-        private ComputeJobContext jCtx;
+        private transient ComputeJobContext jCtx;
 
         /** */
         @LoggerResource
-        private IgniteLogger log;
+        private transient IgniteLogger log;
+
+        /** */
+        transient boolean serialize;
 
         /**
          * @param svcName Service name.
@@ -1898,6 +1929,16 @@ public class GridServiceProcessor extends GridProcessorAdapter {
 
             return serviceTopology(cache, svcName);
         }
+
+        /**
+         * @param self Instance of current class before serialization.
+         * @param ver Sender job version.
+         * @return List of serializable transient fields.
+         */
+        @SuppressWarnings("unused")
+        private static String[] serializableTransient(ServiceTopologyCallable self, IgniteProductVersion ver) {
+            return (self != null && self.serialize) || (ver != null && SERVICE_TOP_CALLABLE_VER1.contains(ver)) ? SER_FIELDS : null;
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/92fff630/modules/core/src/main/java/org/apache/ignite/internal/util/SerializableTransient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/SerializableTransient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/SerializableTransient.java
new file mode 100644
index 0000000..14a2f27
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/SerializableTransient.java
@@ -0,0 +1,58 @@
+/*
+ * 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.util;
+
+import org.apache.ignite.lang.IgniteProductVersion;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Marks class as it has transient fields that should be serialized.
+ * Annotated class must have method that returns list of transient
+ * fields that should be serialized.
+ * <p>
+ *     Works only for jobs. For other messages node version is not available.
+ * </p>
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.TYPE)
+public @interface SerializableTransient {
+    /**
+     * Name of the private static method that returns list of transient fields
+     * that should be serialized (String[]), and accepts itself (before serialization)
+     * and {@link IgniteProductVersion}, e.g.
+     * <pre>
+     *     private static String[] fields(Object self, IgniteProductVersion ver){
+     *         return ver.compareTo("1.5.30") > 0 ? SERIALIZABLE_FIELDS : null;
+     *     }
+     * </pre>
+     * <p>
+     *     On serialization version argument <tt>ver</tt> is null, on deserialization - <tt>self</tt> is null.
+     * </p>
+     * <p>
+     *     If it returns empty array or null all transient fields will be normally
+     *     ignored.
+     * </p>
+     *
+     * @return Name of the method.
+     */
+    String methodName();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/92fff630/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java b/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java
index 9668baf..ad63702 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerUtils.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.marshaller;
 
+import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.marshaller.jdk.JdkMarshaller;
 import org.jetbrains.annotations.Nullable;
 
@@ -24,6 +25,9 @@ import org.jetbrains.annotations.Nullable;
  * Utility marshaller methods.
  */
 public class MarshallerUtils {
+    /** Job sender node version. */
+    private static final ThreadLocal<IgniteProductVersion> JOB_SND_NODE_VER = new ThreadLocal<>();
+
     /**
      * Set node name to marshaller context if possible.
      *
@@ -55,4 +59,22 @@ public class MarshallerUtils {
     private MarshallerUtils() {
         // No-op.
     }
+
+    /**
+     * Sets thread local job sender node version.
+     *
+     * @param ver Thread local job sender node version.
+     */
+    public static void jobSenderVersion(IgniteProductVersion ver) {
+        JOB_SND_NODE_VER.set(ver);
+    }
+
+    /**
+     * Returns thread local job sender node version.
+     *
+     * @return Thread local job sender node version.
+     */
+    public static IgniteProductVersion jobSenderVersion() {
+        return JOB_SND_NODE_VER.get();
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92fff630/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
index 5a5b54d..160f2c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
@@ -47,8 +47,11 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentMap;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.marshaller.MarshallerContext;
 import org.apache.ignite.marshaller.MarshallerExclusions;
+import org.apache.ignite.internal.util.SerializableTransient;
+import org.apache.ignite.marshaller.MarshallerUtils;
 
 import static java.lang.reflect.Modifier.isFinal;
 import static java.lang.reflect.Modifier.isPrivate;
@@ -166,6 +169,9 @@ class OptimizedClassDescriptor {
     /** Proxy interfaces. */
     private Class<?>[] proxyIntfs;
 
+    /** Method returns serializable transient fields. */
+    private Method serTransMtd;
+
     /**
      * Creates descriptor for class.
      *
@@ -441,6 +447,27 @@ class OptimizedClassDescriptor {
 
                         readObjMtds.add(mtd);
 
+                        final SerializableTransient serTransAn = c.getAnnotation(SerializableTransient.class);
+
+                        // Custom serialization policy for transient fields.
+                        if (serTransAn != null) {
+                            try {
+                                serTransMtd = c.getDeclaredMethod(serTransAn.methodName(), cls, IgniteProductVersion.class);
+
+                                int mod = serTransMtd.getModifiers();
+
+                                if (isStatic(mod) && isPrivate(mod)
+                                    && serTransMtd.getReturnType() == String[].class)
+                                    serTransMtd.setAccessible(true);
+                                else
+                                    // Set method back to null if it has incorrect signature.
+                                    serTransMtd = null;
+                            }
+                            catch (NoSuchMethodException ignored) {
+                                serTransMtd = null;
+                            }
+                        }
+
                         Field[] clsFields0 = c.getDeclaredFields();
 
                         Map<String, Field> fieldNames = new HashMap<>();
@@ -797,7 +824,7 @@ class OptimizedClassDescriptor {
                 writeTypeData(out);
 
                 out.writeShort(checksum);
-                out.writeSerializable(obj, writeObjMtds, fields);
+                out.writeSerializable(obj, writeObjMtds, serializableFields(obj.getClass(), obj, null));
 
                 break;
 
@@ -807,6 +834,60 @@ class OptimizedClassDescriptor {
     }
 
     /**
+     * Gets list of serializable fields. If {@link #serTransMtd} method
+     * returns list of transient fields, they will be added to other fields.
+     * Transient fields that are not included in that list will be normally
+     * ignored.
+     *
+     * @param cls Class.
+     * @param obj Object.
+     * @param ver Job sender version.
+     * @return Serializable fields.
+     */
+    @SuppressWarnings("ForLoopReplaceableByForEach")
+    private Fields serializableFields(Class<?> cls, Object obj, IgniteProductVersion ver) {
+        if (serTransMtd == null)
+            return fields;
+
+        try {
+            final String[] transFields = (String[])serTransMtd.invoke(cls, obj, ver);
+
+            if (transFields == null || transFields.length == 0)
+                return fields;
+
+            List<FieldInfo> clsFields = new ArrayList<>();
+
+            clsFields.addAll(fields.fields.get(0).fields);
+
+            for (int i = 0; i < transFields.length; i++) {
+                final String fieldName = transFields[i];
+
+                final Field f = cls.getDeclaredField(fieldName);
+
+                FieldInfo fieldInfo = new FieldInfo(f, f.getName(),
+                    GridUnsafe.objectFieldOffset(f), fieldType(f.getType()));
+
+                clsFields.add(fieldInfo);
+            }
+
+            Collections.sort(clsFields, new Comparator<FieldInfo>() {
+                @Override public int compare(FieldInfo t1, FieldInfo t2) {
+                    return t1.name().compareTo(t2.name());
+                }
+            });
+
+            List<ClassFields> fields = new ArrayList<>();
+
+            fields.add(new ClassFields(clsFields));
+
+            return new Fields(fields);
+        }
+        catch (Exception e) {
+            return fields;
+        }
+    }
+
+    /**
      * @param out Output stream.
      * @throws IOException In case of error.
      */
@@ -838,7 +919,12 @@ class OptimizedClassDescriptor {
             case SERIALIZABLE:
                 verifyChecksum(in.readShort());
 
-                return in.readSerializable(cls, readObjMtds, readResolveMtd, fields);
+                // If no serialize method, then unmarshal as usual.
+                if (serTransMtd != null)
+                    return in.readSerializable(cls, readObjMtds, readResolveMtd,
+                        serializableFields(cls, null, MarshallerUtils.jobSenderVersion()));
+                else
+                    return in.readSerializable(cls, readObjMtds, readResolveMtd, fields);
 
             default:
                 assert false : "Unexpected type: " + type;


[12/50] [abbrv] ignite git commit: IGNITE-4121 .NET: add ScanQuery example

Posted by vo...@apache.org.
IGNITE-4121 .NET: add ScanQuery example


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

Branch: refs/heads/ignite-2693
Commit: d5e15af76044cf65385672f8528d48ecdeca3cb6
Parents: 3a9cbed
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Wed Nov 2 12:02:00 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Wed Nov 2 12:02:00 2016 +0300

----------------------------------------------------------------------
 .../Datagrid/QueryExample.cs                    | 24 +++++++++-
 .../Apache.Ignite.ExamplesDll.csproj            |  1 +
 .../Datagrid/ScanQueryFilter.cs                 | 50 ++++++++++++++++++++
 3 files changed, 73 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d5e15af7/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs
index 1c35149..98f9f50 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs
@@ -19,13 +19,13 @@ namespace Apache.Ignite.Examples.Datagrid
 {
     using System;
     using System.Collections;
-    using System.Collections.Generic;
     using Apache.Ignite.Core;
     using Apache.Ignite.Core.Cache;
     using Apache.Ignite.Core.Cache.Affinity;
     using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Cache.Query;
     using Apache.Ignite.ExamplesDll.Binary;
+    using Apache.Ignite.ExamplesDll.Datagrid;
 
     /// <summary>
     /// This example populates cache with sample data and runs several SQL and
@@ -75,6 +75,9 @@ namespace Apache.Ignite.Examples.Datagrid
                 PopulateCache(employeeCacheColocated);
                 PopulateCache(organizationCache);
 
+                // Run scan query example.
+                ScanQueryExample(employeeCache);
+
                 // Run SQL query example.
                 SqlQueryExample(employeeCache);
 
@@ -102,6 +105,23 @@ namespace Apache.Ignite.Examples.Datagrid
         /// Queries employees that have provided ZIP code in address.
         /// </summary>
         /// <param name="cache">Cache.</param>
+        private static void ScanQueryExample(ICache<int, Employee> cache)
+        {
+            const int zip = 94109;
+
+            var qry = cache.Query(new ScanQuery<int, Employee>(new ScanQueryFilter(zip)));
+
+            Console.WriteLine();
+            Console.WriteLine(">>> Employees with zipcode {0} (scan):", zip);
+
+            foreach (var entry in qry)
+                Console.WriteLine(">>>    " + entry.Value);
+        }
+
+        /// <summary>
+        /// Queries employees that have provided ZIP code in address.
+        /// </summary>
+        /// <param name="cache">Cache.</param>
         private static void SqlQueryExample(ICache<int, Employee> cache)
         {
             const int zip = 94109;
@@ -109,7 +129,7 @@ namespace Apache.Ignite.Examples.Datagrid
             var qry = cache.Query(new SqlQuery(typeof(Employee), "zip = ?", zip));
 
             Console.WriteLine();
-            Console.WriteLine(">>> Employees with zipcode " + zip + ":");
+            Console.WriteLine(">>> Employees with zipcode {0} (SQL):", zip);
 
             foreach (var entry in qry)
                 Console.WriteLine(">>>    " + entry.Value);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d5e15af7/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj
index 41981d8..a41c2f4 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj
@@ -49,6 +49,7 @@
     <Compile Include="Datagrid\EmployeeStorePredicate.cs" />
     <Compile Include="Datagrid\ContinuousQueryFilter.cs" />
     <Compile Include="Datagrid\EmployeeStore.cs" />
+    <Compile Include="Datagrid\ScanQueryFilter.cs" />
     <Compile Include="Events\LocalListener.cs" />
     <Compile Include="Messaging\LocalListener.cs" />
     <Compile Include="Messaging\RemoteOrderedListener.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/d5e15af7/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/ScanQueryFilter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/ScanQueryFilter.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/ScanQueryFilter.cs
new file mode 100644
index 0000000..369b5d0
--- /dev/null
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Datagrid/ScanQueryFilter.cs
@@ -0,0 +1,50 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.ExamplesDll.Datagrid
+{
+    using System;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.ExamplesDll.Binary;
+    
+    /// <summary>
+    /// Filter for scan query example.
+    /// </summary>
+    [Serializable]
+    public class ScanQueryFilter : ICacheEntryFilter<int, Employee>
+    {
+        /** Zip code to filter on. */
+        private readonly int _zipCode;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ScanQueryFilter"/> class.
+        /// </summary>
+        /// <param name="zipCode">The zip code.</param>
+        public ScanQueryFilter(int zipCode)
+        {
+            _zipCode = zipCode;
+        }
+
+        /// <summary>
+        /// Returns a value indicating whether provided cache entry satisfies this predicate.
+        /// </summary>
+        public bool Invoke(ICacheEntry<int, Employee> entry)
+        {
+            return entry.Value.Address.Zip == _zipCode;
+        }
+    }
+}


[22/50] [abbrv] ignite git commit: GG-11360 - Implement SQL queries cancellation Fix for commit 80abd1b: for distributed joins need always send cancel request.

Posted by vo...@apache.org.
GG-11360 - Implement SQL queries cancellation
Fix for commit 80abd1b: for distributed joins need always send cancel request.


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

Branch: refs/heads/ignite-2693
Commit: b1c7c9bb95c900083702d0ba0362edf3aea5a7b4
Parents: a62a013
Author: sboikov <sb...@gridgain.com>
Authored: Mon Nov 7 15:40:36 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Nov 7 15:40:36 2016 +0300

----------------------------------------------------------------------
 .../h2/twostep/GridReduceQueryExecutor.java     | 25 ++++++++++++++------
 1 file changed, 18 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b1c7c9bb/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 3847373..3b851a6 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
@@ -756,7 +756,7 @@ public class GridReduceQueryExecutor {
             }
             finally {
                 // Make sure any activity related to current attempt is cancelled.
-                cancelRemoteQueriesIfNeeded(nodes, r, qryReqId);
+                cancelRemoteQueriesIfNeeded(nodes, r, qryReqId, qry.distributedJoins());
 
                 if (!runs.remove(qryReqId, r))
                     U.warn(log, "Query run was already removed: " + qryReqId);
@@ -793,15 +793,26 @@ public class GridReduceQueryExecutor {
     }
 
     /**
+     * @param nodes Query nodes.
      * @param r Query run.
      * @param qryReqId Query id.
+     * @param distributedJoins Distributed join flag.
      */
-    private void cancelRemoteQueriesIfNeeded(Collection<ClusterNode> nodes, QueryRun r, long qryReqId) {
-        for (GridMergeIndex idx : r.idxs) {
-            if (!idx.fetchedAll()) {
-                send(nodes, new GridQueryCancelRequest(qryReqId), null, false);
-
-                break;
+    private void cancelRemoteQueriesIfNeeded(Collection<ClusterNode> nodes,
+        QueryRun r,
+        long qryReqId,
+        boolean distributedJoins)
+    {
+        // For distributedJoins need always send cancel request to cleanup resources.
+        if (distributedJoins)
+            send(nodes, new GridQueryCancelRequest(qryReqId), null, false);
+        else {
+            for (GridMergeIndex idx : r.idxs) {
+                if (!idx.fetchedAll()) {
+                    send(nodes, new GridQueryCancelRequest(qryReqId), null, false);
+
+                    break;
+                }
             }
         }
     }


[47/50] [abbrv] ignite git commit: IGNITE-4216 .NET: Fix PlatformAffinityFunction to inject resource into baseFunc

Posted by vo...@apache.org.
IGNITE-4216 .NET: Fix PlatformAffinityFunction to inject resource into baseFunc


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

Branch: refs/heads/ignite-2693
Commit: 3aae5cb1d9d547f1d36783f902fdab3e9d5da570
Parents: ad2831e
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Nov 14 13:38:33 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Nov 14 13:38:33 2016 +0300

----------------------------------------------------------------------
 .../affinity/PlatformAffinityFunction.java      |  7 ++++-
 .../dotnet/PlatformDotNetAffinityFunction.java  |  9 ++----
 .../Cache/Affinity/AffinityFunctionTest.cs      | 33 ++++++++++++++++++--
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |  3 +-
 4 files changed, 41 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3aae5cb1/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunction.java
index 1e844e7..8076a19 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunction.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunction.java
@@ -18,10 +18,12 @@
 package org.apache.ignite.internal.processors.platform.cache.affinity;
 
 import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.affinity.AffinityFunction;
 import org.apache.ignite.cache.affinity.AffinityFunctionContext;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
@@ -298,7 +300,10 @@ public class PlatformAffinityFunction implements AffinityFunction, Externalizabl
      */
     @SuppressWarnings("unused")
     @IgniteInstanceResource
-    public void setIgnite(Ignite ignite) {
+    public void setIgnite(Ignite ignite) throws IgniteCheckedException {
         this.ignite = ignite;
+
+        if (baseFunc != null && ignite != null)
+            ((IgniteEx)ignite).context().resource().injectGeneric(baseFunc);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/3aae5cb1/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetAffinityFunction.java
index 483fd22..f9f457d 100644
--- a/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetAffinityFunction.java
+++ b/modules/core/src/main/java/org/apache/ignite/platform/dotnet/PlatformDotNetAffinityFunction.java
@@ -18,17 +18,12 @@
 package org.apache.ignite.platform.dotnet;
 
 import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.binary.BinaryRawWriter;
 import org.apache.ignite.cache.affinity.AffinityFunction;
 import org.apache.ignite.cache.affinity.AffinityFunctionContext;
 import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.binary.BinaryRawWriterEx;
-import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinityFunction;
-import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
-import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
-import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 import org.apache.ignite.lifecycle.LifecycleAware;
 import org.apache.ignite.resources.IgniteInstanceResource;
 
@@ -178,7 +173,7 @@ public class PlatformDotNetAffinityFunction implements AffinityFunction, Externa
      */
     @SuppressWarnings("unused")
     @IgniteInstanceResource
-    private void setIgnite(Ignite ignite) {
+    private void setIgnite(Ignite ignite) throws IgniteCheckedException {
         assert func != null;
 
         func.setIgnite(ignite);

http://git-wip-us.apache.org/repos/asf/ignite/blob/3aae5cb1/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityFunctionTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityFunctionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityFunctionTest.cs
index f38cb3e..d4b6680 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityFunctionTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityFunctionTest.cs
@@ -100,8 +100,8 @@ namespace Apache.Ignite.Core.Tests.Cache.Affinity
         public void FixtureTearDown()
         {
             // Check that affinity handles are present
-            TestUtils.AssertHandleRegistryHasItems(_ignite, _ignite.GetCacheNames().Count - 1, 0);
-            TestUtils.AssertHandleRegistryHasItems(_ignite2, _ignite.GetCacheNames().Count - 1, 0);
+            TestUtils.AssertHandleRegistryHasItems(_ignite, _ignite.GetCacheNames().Count - 3, 0);
+            TestUtils.AssertHandleRegistryHasItems(_ignite2, _ignite.GetCacheNames().Count - 3, 0);
 
             // Destroy all caches
             _ignite.GetCacheNames().ToList().ForEach(_ignite.DestroyCache);
@@ -165,6 +165,34 @@ namespace Apache.Ignite.Core.Tests.Cache.Affinity
         }
 
         /// <summary>
+        /// Tests the dynamic cache with predefined functions.
+        /// </summary>
+        [Test]
+        public void TestDynamicCachePredefined()
+        {
+            var caches = new[]
+            {
+                new CacheConfiguration("rendezvousPredefined")
+                {
+                    AffinityFunction = new RendezvousAffinityFunction {Partitions = 1234}
+                },
+                new CacheConfiguration("fairPredefined")
+                {
+                    AffinityFunction = new FairAffinityFunction {Partitions = 1234}
+                },
+            }.Select(_ignite.CreateCache<int, int>);
+
+            foreach (var cache in caches)
+            {
+                Assert.AreEqual(1234, cache.GetConfiguration().AffinityFunction.Partitions);
+
+                cache[1] = 2;
+
+                Assert.AreEqual(2, cache[1]);
+            }
+        }
+
+        /// <summary>
         /// Verifies the cache affinity.
         /// </summary>
         private static void VerifyCacheAffinity(ICache<int, int> cache)
@@ -231,6 +259,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Affinity
             });
 
             var ex = Assert.Throws<CacheException>(() => cache.Put(1, 2));
+            Assert.IsNotNull(ex.InnerException);
             Assert.AreEqual("User error", ex.InnerException.Message);
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3aae5cb1/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
index 5a9c824..7d483bb 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
@@ -287,7 +287,8 @@ namespace Apache.Ignite.Core.Tests
             var items = handleRegistry.GetItems().Where(x => !(x.Value is LifecycleBeanHolder)).ToList();
 
             if (items.Any())
-                Assert.Fail("HandleRegistry is not empty in grid '{0}':\n '{1}'", grid.Name,
+                Assert.Fail("HandleRegistry is not empty in grid '{0}' (expected {1}, actual {2}):\n '{3}'", 
+                    grid.Name, expectedCount, handleRegistry.Count,
                     items.Select(x => x.ToString()).Aggregate((x, y) => x + "\n" + y));
         }
 


[40/50] [abbrv] ignite git commit: .NET: Fix code analysis warnings in EntityFramework component

Posted by vo...@apache.org.
.NET: Fix code analysis warnings in EntityFramework component


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

Branch: refs/heads/ignite-2693
Commit: c6e0061711b7bc99c87a64f96a3b0ec687aad3a8
Parents: 767a8a3
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Thu Nov 10 11:07:04 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu Nov 10 11:07:04 2016 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.EntityFramework/DbCachingPolicy.cs         | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c6e00617/modules/platforms/dotnet/Apache.Ignite.EntityFramework/DbCachingPolicy.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/DbCachingPolicy.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/DbCachingPolicy.cs
index 9e05ca9..17aa68a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/DbCachingPolicy.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/DbCachingPolicy.cs
@@ -23,6 +23,7 @@ namespace Apache.Ignite.EntityFramework
     /// Default caching policy implementation: everything is cached with <see cref="DbCachingMode.ReadWrite"/>, 
     /// no expiration.
     /// </summary>
+    // ReSharper disable once ClassWithVirtualMembersNeverInherited.Global
     public class DbCachingPolicy : IDbCachingPolicy
     {
         /// <summary>


[15/50] [abbrv] ignite git commit: GG-11655 - Fix merge

Posted by vo...@apache.org.
GG-11655 - Fix merge


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

Branch: refs/heads/ignite-2693
Commit: a62a0136d295486d95c6e2ab5bba88270d831753
Parents: 92fff63
Author: dkarachentsev <dk...@gridgain.com>
Authored: Wed Nov 2 19:07:45 2016 +0300
Committer: dkarachentsev <dk...@gridgain.com>
Committed: Wed Nov 2 19:10:01 2016 +0300

----------------------------------------------------------------------
 .../service/GridServiceProcessor.java           | 136 ++++++++++---------
 1 file changed, 74 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a62a0136/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 8489875..6c26363 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -851,7 +851,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
             }
         }
 
-        return new GridServiceProxy<>(prj, name, svcItf, sticky, ctx).proxy();
+        return new GridServiceProxy<T>(prj, name, svcItf, sticky, ctx).proxy();
     }
 
     /**
@@ -904,7 +904,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
      * @param topVer Topology version.
      * @throws IgniteCheckedException If failed.
      */
-    private void reassign(GridServiceDeployment dep, long topVer) throws IgniteCheckedException {
+    private void reassign(GridServiceDeployment dep, AffinityTopologyVersion topVer) throws IgniteCheckedException {
         ServiceConfiguration cfg = dep.configuration();
 
         Object nodeFilter = cfg.getNodeFilter();
@@ -918,7 +918,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
         Object affKey = cfg.getAffinityKey();
 
         while (true) {
-            GridServiceAssignments assigns = new GridServiceAssignments(cfg, dep.nodeId(), topVer);
+            GridServiceAssignments assigns = new GridServiceAssignments(cfg, dep.nodeId(), topVer.topologyVersion());
 
              Collection<ClusterNode> nodes;
 
@@ -948,7 +948,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                 Map<UUID, Integer> cnts = new HashMap<>();
 
                 if (affKey != null) {
-                    ClusterNode n = ctx.affinity().mapKeyToNode(cacheName, affKey, new AffinityTopologyVersion(topVer));
+                    ClusterNode n = ctx.affinity().mapKeyToNode(cacheName, affKey, topVer);
 
                     if (n != null) {
                         int cnt = maxPerNodeCnt == 0 ? totalCnt == 0 ? 1 : totalCnt : maxPerNodeCnt;
@@ -1180,7 +1180,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
         if (cfg instanceof LazyServiceConfiguration) {
             byte[] bytes = ((LazyServiceConfiguration)cfg).serviceBytes();
 
-            Service srvc = m.unmarshal(bytes, U.resolveClassLoader(null, ctx.config()));
+            Service srvc = U.unmarshal(m, bytes, U.resolveClassLoader(null, ctx.config()));
 
             ctx.resource().inject(srvc);
 
@@ -1190,10 +1190,9 @@ public class GridServiceProcessor extends GridProcessorAdapter {
             Service svc = cfg.getService();
 
             try {
-                byte[] bytes = m.marshal(svc);
+                byte[] bytes = U.marshal(m, svc);
 
-                Service cp = m.unmarshal(bytes,
-                    U.resolveClassLoader(svc.getClass().getClassLoader(), ctx.config()));
+                Service cp = U.unmarshal(m, bytes, U.resolveClassLoader(svc.getClass().getClassLoader(), ctx.config()));
 
                 ctx.resource().inject(cp);
 
@@ -1268,8 +1267,8 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                 ClusterNode oldestSrvNode =
                     CU.oldestAliveCacheServerNode(cache.context().shared(), AffinityTopologyVersion.NONE);
 
-            if (oldestSrvNode == null)
-                return F.emptyIterator();
+                if (oldestSrvNode == null)
+                    return new GridEmptyIterator<>();
 
                 GridCacheQueryManager qryMgr = cache.context().queries();
 
@@ -1455,7 +1454,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
             svcName.set(dep.configuration().getName());
 
             // Ignore other utility cache events.
-            long topVer = ctx.discovery().topologyVersion();
+            AffinityTopologyVersion topVer = ctx.discovery().topologyVersionEx();
 
             ClusterNode oldest = U.oldest(ctx.discovery().nodes(topVer), null);
 
@@ -1506,60 +1505,60 @@ public class GridServiceProcessor extends GridProcessorAdapter {
         }
     }
 
-    /**
-     * Deployment callback.
-     *
-     * @param dep Service deployment.
-     * @param topVer Topology version.
-     */
-    private void onDeployment(final GridServiceDeployment dep, final long topVer) {
-        // Retry forever.
-        try {
-            long newTopVer = ctx.discovery().topologyVersion();
-
-            // If topology version changed, reassignment will happen from topology event.
-            if (newTopVer == topVer)
-                reassign(dep, topVer);
-        }
-        catch (IgniteCheckedException e) {
-            if (!(e instanceof ClusterTopologyCheckedException))
-                log.error("Failed to do service reassignment (will retry): " + dep.configuration().getName(), e);
-
-            long newTopVer = ctx.discovery().topologyVersion();
-
-            if (newTopVer != topVer) {
-                assert newTopVer > topVer;
+        /**
+         * Deployment callback.
+         *
+         * @param dep Service deployment.
+         * @param topVer Topology version.
+         */
+        private void onDeployment(final GridServiceDeployment dep, final AffinityTopologyVersion topVer) {
+            // Retry forever.
+            try {
+                AffinityTopologyVersion newTopVer = ctx.discovery().topologyVersionEx();
 
-                // Reassignment will happen from topology event.
-                return;
+                // If topology version changed, reassignment will happen from topology event.
+                if (newTopVer.equals(topVer))
+                    reassign(dep, topVer);
             }
+            catch (IgniteCheckedException e) {
+                if (!(e instanceof ClusterTopologyCheckedException))
+                    log.error("Failed to do service reassignment (will retry): " + dep.configuration().getName(), e);
 
-            ctx.timeout().addTimeoutObject(new GridTimeoutObject() {
-                private IgniteUuid id = IgniteUuid.randomUuid();
+                AffinityTopologyVersion newTopVer = ctx.discovery().topologyVersionEx();
 
-                private long start = System.currentTimeMillis();
+                if (!newTopVer.equals(topVer)) {
+                    assert newTopVer.compareTo(topVer) > 0;
 
-                @Override public IgniteUuid timeoutId() {
-                    return id;
+                    // Reassignment will happen from topology event.
+                    return;
                 }
 
-                @Override public long endTime() {
-                    return start + RETRY_TIMEOUT;
-                }
+                ctx.timeout().addTimeoutObject(new GridTimeoutObject() {
+                    private IgniteUuid id = IgniteUuid.randomUuid();
 
-                @Override public void onTimeout() {
-                    if (!busyLock.enterBusy())
-                        return;
+                    private long start = System.currentTimeMillis();
 
-                    try {
-                        // Try again.
-                        onDeployment(dep, topVer);
+                    @Override public IgniteUuid timeoutId() {
+                        return id;
                     }
-                    finally {
-                        busyLock.leaveBusy();
+
+                    @Override public long endTime() {
+                        return start + RETRY_TIMEOUT;
                     }
-                }
-            });
+
+                    @Override public void onTimeout() {
+                        if (!busyLock.enterBusy())
+                            return;
+
+                        try {
+                            // Try again.
+                            onDeployment(dep, topVer);
+                        }
+                        finally {
+                            busyLock.leaveBusy();
+                        }
+                    }
+                });
         }
     }
 
@@ -1568,16 +1567,28 @@ public class GridServiceProcessor extends GridProcessorAdapter {
      */
     private class TopologyListener implements GridLocalEventListener {
         /** {@inheritDoc} */
-        @Override public void onEvent(final Event evt) {
+        @Override public void onEvent(Event evt) {
             if (!busyLock.enterBusy())
                 return;
 
             try {
+                final AffinityTopologyVersion topVer;
+
+                if (evt instanceof DiscoveryCustomEvent) {
+                    DiscoveryCustomMessage msg = ((DiscoveryCustomEvent)evt).customMessage();
+
+                    topVer = ((DiscoveryCustomEvent)evt).affinityTopologyVersion();
+
+                    if (msg instanceof CacheAffinityChangeMessage) {
+                        if (!((CacheAffinityChangeMessage)msg).exchangeNeeded())
+                            return;
+                    }
+                }
+                else
+                    topVer = new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion(), 0);
+
                 depExe.submit(new BusyRunnable() {
                     @Override public void run0() {
-                        AffinityTopologyVersion topVer =
-                            new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion());
-
                         ClusterNode oldest = CU.oldestAliveCacheServerNode(cache.context().shared(), topVer);
 
                         if (oldest != null && oldest.isLocal()) {
@@ -1612,7 +1623,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                                         ctx.cache().internalCache(UTILITY_CACHE_NAME).context().affinity().
                                             affinityReadyFuture(topVer).get();
 
-                                        reassign(dep, topVer.topologyVersion());
+                                        reassign(dep, topVer);
                                     }
                                     catch (IgniteCheckedException ex) {
                                         if (!(e instanceof ClusterTopologyCheckedException))
@@ -1629,7 +1640,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                             }
 
                             if (!retries.isEmpty())
-                                onReassignmentFailed(topVer.topologyVersion(), retries);
+                                onReassignmentFailed(topVer, retries);
                         }
 
                         // Clean up zombie assignments.
@@ -1666,13 +1677,14 @@ public class GridServiceProcessor extends GridProcessorAdapter {
          * @param topVer Topology version.
          * @param retries Retries.
          */
-        private void onReassignmentFailed(final long topVer, final Collection<GridServiceDeployment> retries) {
+        private void onReassignmentFailed(final AffinityTopologyVersion topVer,
+            final Collection<GridServiceDeployment> retries) {
             if (!busyLock.enterBusy())
                 return;
 
             try {
                 // If topology changed again, let next event handle it.
-                if (ctx.discovery().topologyVersion() != topVer)
+                if (ctx.discovery().topologyVersionEx().equals(topVer))
                     return;
 
                 for (Iterator<GridServiceDeployment> it = retries.iterator(); it.hasNext(); ) {


[26/50] [abbrv] ignite git commit: IGNITE-4185 .NET: Fix NullReferenceException in IgniteOutputCacheProvider when igniteConfiguration is missing

Posted by vo...@apache.org.
IGNITE-4185 .NET: Fix NullReferenceException in IgniteOutputCacheProvider when igniteConfiguration is missing


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

Branch: refs/heads/ignite-2693
Commit: f697fb5786fb4ce15f581c465ff0dcb3d2bb7b14
Parents: 1612b6d
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Nov 8 19:13:48 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Nov 8 19:13:48 2016 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.AspNet.Tests/App.config       |  1 +
 .../IgniteSessionStateStoreProviderTest.cs      | 21 +++++++++++++++++---
 .../Apache.Ignite.AspNet/Impl/ConfigUtil.cs     |  5 +++++
 3 files changed, 24 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f697fb57/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/App.config
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/App.config b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/App.config
index 86ee3d4..7d2c1d0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/App.config
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/App.config
@@ -22,6 +22,7 @@
         <section name="igniteConfiguration" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
         <section name="igniteConfiguration2" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
         <section name="igniteConfiguration3" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
+        <section name="igniteConfigurationInvalid" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
     </configSections>
 
     <runtime>

http://git-wip-us.apache.org/repos/asf/ignite/blob/f697fb57/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
index 2c73359..9c3b07c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
@@ -19,6 +19,7 @@ namespace Apache.Ignite.AspNet.Tests
 {
     using System;
     using System.Collections.Specialized;
+    using System.Configuration;
     using System.Linq;
     using System.Reflection;
     using System.Threading;
@@ -116,14 +117,28 @@ namespace Apache.Ignite.AspNet.Tests
             Assert.Throws<InvalidOperationException>(() =>
                     stateProvider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions));
 
-            // Invalid section.
-            Assert.Throws<IgniteException>(() =>
+            // Missing section.
+            var ex = Assert.Throws<IgniteException>(() =>
                 stateProvider.Initialize("testName", new NameValueCollection
                 {
-                    {SectionNameAttr, "invalidSection"},
+                    {SectionNameAttr, "missingSection"},
                     {CacheNameAttr, CacheName}
                 }));
 
+            Assert.IsInstanceOf<ConfigurationErrorsException>(ex.InnerException);
+
+            // Invalid section with missing content.
+            stateProvider = new IgniteSessionStateStoreProvider();
+
+            ex = Assert.Throws<IgniteException>(() =>
+                stateProvider.Initialize("testName", new NameValueCollection
+                {
+                    {SectionNameAttr, "igniteConfigurationInvalid"},
+                    {CacheNameAttr, CacheName}
+                }));
+
+            Assert.IsInstanceOf<ConfigurationErrorsException>(ex.InnerException);
+
             // Valid grid.
             stateProvider = GetProvider();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f697fb57/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs
index a162d81..fc93c7e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs
@@ -86,6 +86,11 @@ namespace Apache.Ignite.AspNet.Impl
                         "Could not find {0} with name '{1}'", typeof(IgniteConfigurationSection).Name, sectionName));
 
                 config = section.IgniteConfiguration;
+
+                if (config == null)
+                    throw new ConfigurationErrorsException(string.Format(CultureInfo.InvariantCulture,
+                        "{0} with name '{1}' is defined in <configSections>, but not present in configuration", 
+                        typeof(IgniteConfigurationSection).Name, sectionName));
             }
             else
                 config = new IgniteConfiguration {GridName = gridName};


[27/50] [abbrv] ignite git commit: .NET: Fix error messages when IgniteConfigurationSection content is missing

Posted by vo...@apache.org.
.NET: Fix error messages when IgniteConfigurationSection content is missing


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

Branch: refs/heads/ignite-2693
Commit: 69487f2c375010737311af65750a519b403fc17f
Parents: f697fb5
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Nov 8 19:38:28 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Nov 8 19:38:28 2016 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.AspNet/Impl/ConfigUtil.cs     |  2 +-
 .../IgniteConfigurationSectionTest.cs           | 20 ++++++++++++++++++++
 .../dotnet/Apache.Ignite.Core.Tests/app.config  |  1 +
 .../Apache.Ignite.Core.Tests/custom_app.config  |  1 +
 .../dotnet/Apache.Ignite.Core/Ignition.cs       | 12 ++++++++++++
 5 files changed, 35 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/69487f2c/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs
index fc93c7e..b967518 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs
@@ -89,7 +89,7 @@ namespace Apache.Ignite.AspNet.Impl
 
                 if (config == null)
                     throw new ConfigurationErrorsException(string.Format(CultureInfo.InvariantCulture,
-                        "{0} with name '{1}' is defined in <configSections>, but not present in configuration", 
+                        "{0} with name '{1}' is defined in <configSections>, but not present in configuration.", 
                         typeof(IgniteConfigurationSection).Name, sectionName));
             }
             else

http://git-wip-us.apache.org/repos/asf/ignite/blob/69487f2c/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSectionTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSectionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSectionTest.cs
index 4e10a2b..13bd9a2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSectionTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSectionTest.cs
@@ -78,6 +78,7 @@ namespace Apache.Ignite.Core.Tests
         [Test]
         public void TestIgniteStartError()
         {
+            // Missing section in default file.
             var ex = Assert.Throws<ConfigurationErrorsException>(() =>
                 Ignition.StartFromApplicationConfiguration("igniteConfiguration111"));
 
@@ -85,17 +86,36 @@ namespace Apache.Ignite.Core.Tests
                 ex.Message);
 
 
+            // Missing section body.
+            ex = Assert.Throws<ConfigurationErrorsException>(() =>
+                Ignition.StartFromApplicationConfiguration("igniteConfigurationMissing"));
+
+            Assert.AreEqual("IgniteConfigurationSection with name 'igniteConfigurationMissing' " +
+                            "is defined in <configSections>, but not present in configuration.", ex.Message);
+
+
+            // Missing custom file.
             ex = Assert.Throws<ConfigurationErrorsException>(() =>
                 Ignition.StartFromApplicationConfiguration("igniteConfiguration", "somefile"));
 
             Assert.AreEqual("Specified config file does not exist: somefile", ex.Message);
 
 
+            // Missing section in custom file.
             ex = Assert.Throws<ConfigurationErrorsException>(() =>
                 Ignition.StartFromApplicationConfiguration("igniteConfiguration", "custom_app.config"));
 
             Assert.AreEqual("Could not find IgniteConfigurationSection with name 'igniteConfiguration' " +
                             "in file 'custom_app.config'", ex.Message);
+            
+            
+            // Missing section body in custom file.
+            ex = Assert.Throws<ConfigurationErrorsException>(() =>
+                Ignition.StartFromApplicationConfiguration("igniteConfigurationMissing", "custom_app.config"));
+
+            Assert.AreEqual("IgniteConfigurationSection with name 'igniteConfigurationMissing' in file " +
+                            "'custom_app.config' is defined in <configSections>, but not present in configuration.",
+                ex.Message);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/69487f2c/modules/platforms/dotnet/Apache.Ignite.Core.Tests/app.config
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/app.config b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/app.config
index d22bea4..f928ed4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/app.config
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/app.config
@@ -21,6 +21,7 @@
     <configSections>
         <section name="igniteConfiguration" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
         <section name="igniteConfiguration2" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
+        <section name="igniteConfigurationMissing" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
     </configSections>
 
     <runtime>

http://git-wip-us.apache.org/repos/asf/ignite/blob/69487f2c/modules/platforms/dotnet/Apache.Ignite.Core.Tests/custom_app.config
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/custom_app.config b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/custom_app.config
index 41ea39e..aec91dc 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/custom_app.config
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/custom_app.config
@@ -20,6 +20,7 @@
 <configuration>
     <configSections>
         <section name="igniteConfiguration3" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
+        <section name="igniteConfigurationMissing" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
     </configSections>
 
     <igniteConfiguration3 gridName="myGrid3" localhost="127.0.0.1">

http://git-wip-us.apache.org/repos/asf/ignite/blob/69487f2c/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
index 7b023f3..515902f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
@@ -152,6 +152,12 @@ namespace Apache.Ignite.Core
                 throw new ConfigurationErrorsException(string.Format("Could not find {0} with name '{1}'",
                     typeof(IgniteConfigurationSection).Name, sectionName));
 
+            if (section.IgniteConfiguration == null)
+                throw new ConfigurationErrorsException(
+                    string.Format("{0} with name '{1}' is defined in <configSections>, " +
+                                  "but not present in configuration.",
+                        typeof(IgniteConfigurationSection).Name, sectionName));
+
             return Start(section.IgniteConfiguration);
         }
 
@@ -177,6 +183,12 @@ namespace Apache.Ignite.Core
                     string.Format("Could not find {0} with name '{1}' in file '{2}'",
                         typeof(IgniteConfigurationSection).Name, sectionName, configPath));
 
+            if (section.IgniteConfiguration == null)
+                throw new ConfigurationErrorsException(
+                    string.Format("{0} with name '{1}' in file '{2}' is defined in <configSections>, " +
+                                  "but not present in configuration.",
+                        typeof(IgniteConfigurationSection).Name, sectionName, configPath));
+
             return Start(section.IgniteConfiguration);
         }
 


[44/50] [abbrv] ignite git commit: Fixed classnames.properties generation for ignite-hadoop module.

Posted by vo...@apache.org.
Fixed classnames.properties generation for ignite-hadoop module.


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

Branch: refs/heads/ignite-2693
Commit: c6921a311f437504a45a4667ddde85b14269ba57
Parents: 884b281
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Nov 11 16:01:33 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Nov 11 16:01:33 2016 +0700

----------------------------------------------------------------------
 modules/hadoop/pom.xml | 49 +++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 49 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c6921a31/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index c02a574..d0b0481 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -127,6 +127,55 @@
     <build>
         <plugins>
             <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>exec-maven-plugin</artifactId>
+                <version>1.3.2</version>
+                <dependencies>
+                    <dependency>
+                        <groupId>org.apache.ignite</groupId>
+                        <artifactId>ignite-tools</artifactId>
+                        <version>${project.version}</version>
+                    </dependency>
+                </dependencies>
+                <executions>
+                    <execution>
+                        <phase>process-classes</phase>
+                        <goals>
+                            <goal>java</goal>
+                        </goals>
+                        <configuration>
+                            <includePluginDependencies>true</includePluginDependencies>
+                            <mainClass>org.apache.ignite.tools.classgen.ClassesGenerator</mainClass>
+                            <arguments>
+                                <argument>${project.basedir}/target/classes</argument>
+                                <argument>
+                                    <![CDATA[
+#
+# 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.
+#
+                                    ]]>
+                                </argument>
+                                <argument>org.apache.ignite.hadoop:org.apache.ignite.internal.processors.hadoop</argument>
+                            </arguments>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+
+            <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-jar-plugin</artifactId>
                 <version>2.2</version>


[30/50] [abbrv] ignite git commit: Fixes after merge.

Posted by vo...@apache.org.
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/703957dd
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/703957dd
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/703957dd

Branch: refs/heads/ignite-2693
Commit: 703957dd2fba4527a9070ba31cefc38a5f24e490
Parents: 44d0790
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Nov 9 11:24:04 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Nov 9 11:24:04 2016 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/marshaller/MarshallerContextSelfTest.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/703957dd/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextSelfTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextSelfTest.java
index f61a2aa..8a0ff9a 100644
--- a/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextSelfTest.java
@@ -39,7 +39,7 @@ public class MarshallerContextSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testClassName() throws Exception {
-        File workDir = U.resolveWorkDirectory("marshaller", false);
+        File workDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), "marshaller", false);
 
         final MarshallerContextImpl.ContinuousQueryListener queryListener =
                 new MarshallerContextImpl.ContinuousQueryListener(log, workDir);
@@ -67,7 +67,7 @@ public class MarshallerContextSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testOnUpdated() throws Exception {
-        File workDir = U.resolveWorkDirectory("marshaller", false);
+        File workDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), "marshaller", false);
 
         final MarshallerContextImpl.ContinuousQueryListener queryListener =
                 new MarshallerContextImpl.ContinuousQueryListener(log, workDir);


[42/50] [abbrv] ignite git commit: IGNITE-4186 .NET: Fix "Invalid session release request" exception in IgniteSessionStateStoreProvider.SetAndReleaseItemExclusive

Posted by vo...@apache.org.
IGNITE-4186 .NET: Fix "Invalid session release request" exception in IgniteSessionStateStoreProvider.SetAndReleaseItemExclusive

This closes #1227


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

Branch: refs/heads/ignite-2693
Commit: e007f6e35efc2e0cdab94e39510b2dbd77c6e454
Parents: aacdcba
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Thu Nov 10 20:35:15 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu Nov 10 20:35:15 2016 +0300

----------------------------------------------------------------------
 .../IgniteSessionStateStoreProviderTest.cs      | 49 +++++++++++++++++---
 .../IgniteSessionStateStoreProvider.cs          | 21 ++++++---
 2 files changed, 57 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e007f6e3/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
index 9c3b07c..c6e3b30 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
@@ -209,6 +209,24 @@ namespace Apache.Ignite.AspNet.Tests
             Assert.AreEqual(TimeSpan.Zero, lockAge);
             Assert.AreEqual(SessionStateActions.None, actions);
 
+            // SetAndRelease with no lock. This happens with certain versions of ASP.NET.
+            var item = provider.CreateNewStoreData(HttpContext, 7);
+            // ReSharper disable once AssignNullToNotNullAttribute (lockId is not supposed to be null, but it can be).
+            provider.SetAndReleaseItemExclusive(HttpContext, Id, item, null, true);
+
+            // Check added item.
+            res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions);
+            Assert.IsNotNull(res);
+            Assert.IsNull(lockId);
+            Assert.AreEqual(7, res.Timeout);
+            Assert.IsFalse(locked);
+            Assert.AreEqual(TimeSpan.Zero, lockAge);
+            Assert.AreEqual(SessionStateActions.None, actions);
+
+            // Remove item.
+            // ReSharper disable once AssignNullToNotNullAttribute (lockId is not supposed to be null, but it can be).
+            provider.RemoveItem(HttpContext, Id, null, null);
+
             // Add item.
             provider.CreateUninitializedItem(HttpContext, Id, 7);
             
@@ -228,7 +246,7 @@ namespace Apache.Ignite.AspNet.Tests
             Assert.IsFalse(locked);
             Assert.AreEqual(TimeSpan.Zero, lockAge);
             Assert.AreEqual(SessionStateActions.None, actions);
-            provider.SetAndReleaseItemExclusive(HttpContext, Id, UpdateStoreData(res), lockId, true);
+            provider.SetAndReleaseItemExclusive(HttpContext, Id, UpdateStoreData(res), lockId, false);
 
             // Not locked, item present.
             res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions);
@@ -331,20 +349,37 @@ namespace Apache.Ignite.AspNet.Tests
             Assert.IsFalse(GetProvider().SetItemExpireCallback(null));
 
             // Check there is no item.
-            var res = provider.GetItem(HttpContext, "myId", out locked, out lockAge, out lockId, out actions);
+            var res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions);
             Assert.IsNull(res);
 
-            // Put an item.
-            provider.CreateUninitializedItem(HttpContext, "myId", 1);
+            // Put an item with CreateUninitializedItem and check.
+            provider.CreateUninitializedItem(HttpContext, Id, 1);
+            CheckExpiry(provider);
+
+            // Put an item with SetAndReleaseItemExclusive and check.
+            var data = provider.CreateNewStoreData(HttpContext, 1);
+            provider.SetAndReleaseItemExclusive(HttpContext, Id, data, lockId, true);
+            CheckExpiry(provider);
+        }
 
-            // Check that it is there.
-            res = provider.GetItem(HttpContext, "myId", out locked, out lockAge, out lockId, out actions);
+        /// <summary>
+        /// Checks item expiration.
+        /// </summary>
+        private static void CheckExpiry(SessionStateStoreProviderBase provider)
+        {
+            bool locked;
+            TimeSpan lockAge;
+            object lockId;
+            SessionStateActions actions;
+
+            // Check that item is present.
+            var res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions);
             Assert.IsNotNull(res);
 
             // Wait a minute and check again.
             Thread.Sleep(TimeSpan.FromMinutes(1.05));
 
-            res = provider.GetItem(HttpContext, "myId", out locked, out lockAge, out lockId, out actions);
+            res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions);
             Assert.IsNull(res);
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/e007f6e3/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs
index 86035dd..a5e8199 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs
@@ -294,11 +294,20 @@ namespace Apache.Ignite.AspNet
 
             var data = (IgniteSessionStateStoreData) item;
 
-            if (!(lockId is long) || data.LockId != (long) lockId)
-                throw new IgniteException(string.Format(CultureInfo.InvariantCulture,
-                    "Invalid session release request, expected lockId: {0}, actual: {1}", data.LockId, lockId));
+            if (newItem)
+            {
+                var cache = _expiryCacheHolder.GetCacheWithExpiry(data.Timeout * 60);
+
+                PutItem(key, data, cache);
+            }
+            else
+            {
+                if (!(lockId is long) || data.LockId != (long) lockId)
+                    throw new IgniteException(string.Format(CultureInfo.InvariantCulture,
+                        "Invalid session release request, expected lockId: {0}, actual: {1}", data.LockId, lockId));
 
-            SetAndUnlockItem(key, data);
+                SetAndUnlockItem(key, data);
+            }
         }
 
         /// <summary>
@@ -333,7 +342,7 @@ namespace Apache.Ignite.AspNet
         /// </summary>
         /// <param name="context">The <see cref="T:System.Web.HttpContext" /> for the current request.</param>
         /// <param name="timeout">The session-state <see cref="P:System.Web.SessionState.HttpSessionState.Timeout" /> 
-        /// value for the new <see cref="T:System.Web.SessionState.SessionStateStoreData" />.</param>
+        /// value for the new <see cref="T:System.Web.SessionState.SessionStateStoreData" />, in minutes.</param>
         /// <returns>
         /// A new <see cref="T:System.Web.SessionState.SessionStateStoreData" /> for the current request.
         /// </returns>
@@ -349,7 +358,7 @@ namespace Apache.Ignite.AspNet
         /// <param name="id">The <see cref="P:System.Web.SessionState.HttpSessionState.SessionID" /> 
         /// for the current request.</param>
         /// <param name="timeout">The session <see cref="P:System.Web.SessionState.HttpSessionState.Timeout" /> 
-        /// for the current request.</param>
+        /// for the current request, in minutes.</param>
         public override void CreateUninitializedItem(HttpContext context, string id, int timeout)
         {
             var cache = _expiryCacheHolder.GetCacheWithExpiry((long) timeout * 60);


[06/50] [abbrv] ignite git commit: IGNITE-3368 .NET: Improve test coverage

Posted by vo...@apache.org.
IGNITE-3368 .NET: Improve test coverage

This closes #953

# Conflicts:
#	modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs


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

Branch: refs/heads/ignite-2693
Commit: dc0adf641f0ddac19af84ba0d701b33ee520b067
Parents: 4495efc
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Aug 16 18:11:34 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Nov 1 18:43:05 2016 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.Tests.csproj             |  10 +
 .../Binary/BinaryBuilderSelfTest.cs             |  24 ++-
 .../Binary/BinaryReaderWriterTest.cs            | 171 +++++++++++++++++
 .../Binary/IO/BinaryStreamsTest.cs              | 151 +++++++++++++++
 .../Cache/Affinity/AffinityKeyTest.cs           |  66 +++++++
 .../Affinity/AffinityTopologyVersionTest.cs     |  59 ++++++
 .../Cache/CacheAbstractTest.cs                  |  20 +-
 .../Cache/CacheResultTest.cs                    |  75 ++++++++
 .../Continuous/ContinuousQueryAbstractTest.cs   |  42 ++--
 .../Cache/Store/CacheParallelLoadStoreTest.cs   |   2 +-
 .../Cache/Store/CacheStoreAdapterTest.cs        |  90 +++++++++
 .../Cache/Store/CacheTestParallelLoadStore.cs   |   9 +
 .../Collections/MultiValueDictionaryTest.cs     |  58 ++++++
 .../Collections/ReadOnlyCollectionTest.cs       |  59 ++++++
 .../Collections/ReadOnlyDictionaryTest.cs       |  70 +++++++
 .../Common/IgniteGuidTest.cs                    |  62 ++++++
 .../Compute/AbstractTaskTest.cs                 |  40 ++--
 .../Compute/BinarizableClosureTaskTest.cs       |  18 +-
 .../Compute/BinarizableTaskTest.cs              |  18 +-
 .../Compute/CancellationTest.cs                 |  10 +
 .../Compute/ClosureTaskTest.cs                  | 192 +++++--------------
 .../Compute/ComputeApiTest.cs                   |  53 +++--
 .../Compute/FailoverTaskSelfTest.cs             |   5 +-
 .../Compute/IgniteExceptionTaskSelfTest.cs      |  43 ++---
 .../Compute/ResourceTaskTest.cs                 |  51 +++++
 .../Compute/SerializableClosureTaskTest.cs      |   5 +
 .../Compute/TaskAdapterTest.cs                  |   5 +-
 .../Compute/TaskResultTest.cs                   |  14 +-
 .../Config/Compute/compute-standalone.xml       |   1 +
 .../Apache.Ignite.Core.Tests/EventsTest.cs      | 192 +++++++++++++++----
 .../Examples/ExamplesTest.cs                    |   2 +-
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |  46 +++++
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |   3 +
 .../Apache.Ignite.Core.csproj                   |   3 -
 .../Cache/Affinity/AffinityKey.cs               |  11 ++
 .../Cache/CachePartialUpdateException.cs        |   6 +-
 .../Store/CacheParallelLoadStoreAdapter.cs      |   7 +
 .../Impl/Binary/BinaryObject.cs                 |   2 +
 .../Impl/Binary/BinaryObjectBuilder.cs          |   2 +-
 .../Impl/Binary/BinaryObjectHandle.cs           |  59 ------
 .../Impl/Binary/BinaryObjectHeader.cs           |   5 +-
 .../Binary/BinarySurrogateTypeDescriptor.cs     |   2 +-
 .../Impl/Binary/BinaryWriter.cs                 |  20 --
 .../Impl/Binary/Io/BinaryStreamAdapter.cs       |   5 +
 .../Impl/Binary/Io/BinaryStreamBase.cs          |  11 +-
 .../Impl/Binary/Io/IBinaryStream.cs             |   2 +-
 .../Apache.Ignite.Core/Impl/Binary/JavaTypes.cs |  11 --
 .../Cache/Event/JavaCacheEntryEventFilter.cs    |   2 +
 .../Impl/Collections/MultiValueDictionary.cs    |  26 ---
 .../Impl/Collections/ReadOnlyDictionary.cs      |   2 +-
 .../Impl/Common/ResizeableArray.cs              |  64 -------
 .../Impl/Events/EventTypeConverter.cs           |   3 +
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |   3 +
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |   1 +
 .../Apache.Ignite.Core/Impl/Handle/Handle.cs    |  12 +-
 .../Impl/Handle/HandleRegistry.cs               |  20 +-
 .../Apache.Ignite.Core/Impl/Handle/IHandle.cs   |   5 -
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |   1 +
 .../Apache.Ignite.Core/Impl/IgniteUtils.cs      |  22 +--
 .../Impl/InteropExceptionHolder.cs              |  88 ---------
 .../Impl/Memory/PlatformMemoryUtils.cs          |  30 ---
 .../Impl/Memory/PlatformRawMemory.cs            |   4 +
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |   8 -
 63 files changed, 1420 insertions(+), 683 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 11a8a15..34d93d7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -62,6 +62,16 @@
     <Reference Include="System.Xml.Linq" />
   </ItemGroup>
   <ItemGroup>
+    <Compile Include="Binary\BinaryReaderWriterTest.cs" />
+    <Compile Include="Binary\IO\BinaryStreamsTest.cs" />
+    <Compile Include="Cache\Affinity\AffinityKeyTest.cs" />
+    <Compile Include="Cache\Affinity\AffinityTopologyVersionTest.cs" />
+    <Compile Include="Cache\CacheResultTest.cs" />
+    <Compile Include="Cache\Store\CacheStoreAdapterTest.cs" />
+    <Compile Include="Collections\MultiValueDictionaryTest.cs" />
+    <Compile Include="Collections\ReadOnlyCollectionTest.cs" />
+    <Compile Include="Collections\ReadOnlyDictionaryTest.cs" />
+    <Compile Include="Common\IgniteGuidTest.cs" />
     <Compile Include="Log\DefaultLoggerTest.cs" />
     <Compile Include="Log\Log4NetLoggerTest.cs" />
     <Compile Include="Log\NLogLoggerTest.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs
index 80788a6..c280255 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs
@@ -24,6 +24,7 @@ namespace Apache.Ignite.Core.Tests.Binary
     using System.Collections;
     using System.Collections.Generic;
     using System.Linq;
+    using System.Text.RegularExpressions;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Impl.Binary;
@@ -590,6 +591,23 @@ namespace Apache.Ignite.Core.Tests.Binary
         }
 
         /// <summary>
+        /// Tests equality and formatting members.
+        /// </summary>
+        [Test]
+        public void TestEquality()
+        {
+            var bin = _grid.GetBinary();
+
+            var obj1 = bin.GetBuilder("myType").SetStringField("str", "foo").SetIntField("int", 1).Build();
+            var obj2 = bin.GetBuilder("myType").SetStringField("str", "foo").SetIntField("int", 1).Build();
+
+            Assert.AreEqual(obj1, obj2);
+            Assert.AreEqual(obj1.GetHashCode(), obj2.GetHashCode());
+
+            Assert.IsTrue(Regex.IsMatch(obj1.ToString(), @"myType \[idHash=[0-9]+, str=foo, int=1\]"));
+        }
+
+        /// <summary>
         /// Test primitive fields setting.
         /// </summary>
         [Test]
@@ -1617,9 +1635,13 @@ namespace Apache.Ignite.Core.Tests.Binary
             foreach (var binEnum in binEnums)
             {
                 Assert.IsTrue(binEnum.GetBinaryType().IsEnum);
+
                 Assert.AreEqual(val, binEnum.EnumValue);
-                Assert.AreEqual((TestEnumRegistered)val, binEnum.Deserialize<TestEnumRegistered>());
+
+                Assert.AreEqual((TestEnumRegistered) val, binEnum.Deserialize<TestEnumRegistered>());
             }
+
+            Assert.AreEqual(binEnums[0], binEnums[1]);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryReaderWriterTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryReaderWriterTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryReaderWriterTest.cs
new file mode 100644
index 0000000..e4cff1b
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryReaderWriterTest.cs
@@ -0,0 +1,171 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Binary
+{
+    using System;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Impl.Binary;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests the <see cref="Impl.Binary.BinaryReader"/> and <see cref="Impl.Binary.BinaryWriter"/> classes.
+    /// </summary>
+    public class BinaryReaderWriterTest
+    {
+        /// <summary>
+        /// Tests all read/write methods.
+        /// </summary>
+        [Test]
+        public void TestWriteRead()
+        {
+            var marsh = new Marshaller(new BinaryConfiguration(typeof(ReadWriteAll)));
+
+            marsh.Unmarshal<ReadWriteAll>(marsh.Marshal(new ReadWriteAll()));
+        }
+
+        private class ReadWriteAll : IBinarizable
+        {
+            private static readonly DateTime Date = DateTime.UtcNow;
+
+            private static readonly Guid Guid = Guid.NewGuid();
+
+            public void WriteBinary(IBinaryWriter writer)
+            {
+                writer.WriteByte("Byte", 1);
+                writer.WriteByteArray("ByteArray", new byte[] {1});
+                writer.WriteChar("Char", '1');
+                writer.WriteCharArray("CharArray", new[] {'1'});
+                writer.WriteShort("Short", 1);
+                writer.WriteShortArray("ShortArray", new short[] {1});
+                writer.WriteInt("Int", 1);
+                writer.WriteIntArray("IntArray", new[] {1});
+                writer.WriteLong("Long", 1);
+                writer.WriteLongArray("LongArray", new long[] {1});
+                writer.WriteBoolean("Boolean", true);
+                writer.WriteBooleanArray("BooleanArray", new[] {true});
+                writer.WriteFloat("Float", 1);
+                writer.WriteFloatArray("FloatArray", new float[] {1});
+                writer.WriteDouble("Double", 1);
+                writer.WriteDoubleArray("DoubleArray", new double[] {1});
+                writer.WriteDecimal("Decimal", 1);
+                writer.WriteDecimalArray("DecimalArray", new decimal?[] {1});
+                writer.WriteTimestamp("Timestamp", Date);
+                writer.WriteTimestampArray("TimestampArray", new DateTime?[] {Date});
+                writer.WriteString("String", "1");
+                writer.WriteStringArray("StringArray", new[] {"1"});
+                writer.WriteGuid("Guid", Guid);
+                writer.WriteGuidArray("GuidArray", new Guid?[] {Guid});
+                writer.WriteEnum("Enum", MyEnum.Bar);
+                writer.WriteEnumArray("EnumArray", new[] {MyEnum.Bar});
+
+                var raw = writer.GetRawWriter();
+
+                raw.WriteByte(1);
+                raw.WriteByteArray(new byte[] {1});
+                raw.WriteChar('1');
+                raw.WriteCharArray(new[] {'1'});
+                raw.WriteShort(1);
+                raw.WriteShortArray(new short[] {1});
+                raw.WriteInt(1);
+                raw.WriteIntArray(new[] {1});
+                raw.WriteLong(1);
+                raw.WriteLongArray(new long[] {1});
+                raw.WriteBoolean(true);
+                raw.WriteBooleanArray(new[] {true});
+                raw.WriteFloat(1);
+                raw.WriteFloatArray(new float[] {1});
+                raw.WriteDouble(1);
+                raw.WriteDoubleArray(new double[] {1});
+                raw.WriteDecimal(1);
+                raw.WriteDecimalArray(new decimal?[] {1});
+                raw.WriteTimestamp(Date);
+                raw.WriteTimestampArray(new DateTime?[] {Date});
+                raw.WriteString("1");
+                raw.WriteStringArray(new[] {"1"});
+                raw.WriteGuid(Guid);
+                raw.WriteGuidArray(new Guid?[] {Guid});
+                raw.WriteEnum(MyEnum.Bar);
+                raw.WriteEnumArray(new[] {MyEnum.Bar});
+            }
+
+            public void ReadBinary(IBinaryReader reader)
+            {
+                Assert.AreEqual(1, reader.ReadByte("Byte"));
+                Assert.AreEqual(new byte[] {1}, reader.ReadByteArray("ByteArray"));
+                Assert.AreEqual('1', reader.ReadChar("Char"));
+                Assert.AreEqual(new[] {'1'}, reader.ReadCharArray("CharArray"));
+                Assert.AreEqual(1, reader.ReadShort("Short"));
+                Assert.AreEqual(new short[] {1}, reader.ReadShortArray("ShortArray"));
+                Assert.AreEqual(1, reader.ReadInt("Int"));
+                Assert.AreEqual(new[] {1}, reader.ReadIntArray("IntArray"));
+                Assert.AreEqual(1, reader.ReadLong("Long"));
+                Assert.AreEqual(new long[] {1}, reader.ReadLongArray("LongArray"));
+                Assert.AreEqual(true, reader.ReadBoolean("Boolean"));
+                Assert.AreEqual(new[] {true}, reader.ReadBooleanArray("BooleanArray"));
+                Assert.AreEqual(1, reader.ReadFloat("Float"));
+                Assert.AreEqual(new float[] {1}, reader.ReadFloatArray("FloatArray"));
+                Assert.AreEqual(1, reader.ReadDouble("Double"));
+                Assert.AreEqual(new double[] {1}, reader.ReadDoubleArray("DoubleArray"));
+                Assert.AreEqual(1, reader.ReadDecimal("Decimal"));
+                Assert.AreEqual(new decimal?[] {1}, reader.ReadDecimalArray("DecimalArray"));
+                Assert.AreEqual(Date, reader.ReadTimestamp("Timestamp"));
+                Assert.AreEqual(new DateTime?[] {Date}, reader.ReadTimestampArray("TimestampArray"));
+                Assert.AreEqual("1", reader.ReadString("String"));
+                Assert.AreEqual(new[] {"1"}, reader.ReadStringArray("StringArray"));
+                Assert.AreEqual(Guid, reader.ReadGuid("Guid"));
+                Assert.AreEqual(new Guid?[] {Guid}, reader.ReadGuidArray("GuidArray"));
+                Assert.AreEqual(MyEnum.Bar, reader.ReadEnum<MyEnum>("Enum"));
+                Assert.AreEqual(new[] {MyEnum.Bar}, reader.ReadEnumArray<MyEnum>("EnumArray"));
+
+                var raw = reader.GetRawReader();
+
+                Assert.AreEqual(1, raw.ReadByte());
+                Assert.AreEqual(new byte[] { 1 }, raw.ReadByteArray());
+                Assert.AreEqual('1', raw.ReadChar());
+                Assert.AreEqual(new[] { '1' }, raw.ReadCharArray());
+                Assert.AreEqual(1, raw.ReadShort());
+                Assert.AreEqual(new short[] { 1 }, raw.ReadShortArray());
+                Assert.AreEqual(1, raw.ReadInt());
+                Assert.AreEqual(new[] { 1 }, raw.ReadIntArray());
+                Assert.AreEqual(1, raw.ReadLong());
+                Assert.AreEqual(new long[] { 1 }, raw.ReadLongArray());
+                Assert.AreEqual(true, raw.ReadBoolean());
+                Assert.AreEqual(new[] { true }, raw.ReadBooleanArray());
+                Assert.AreEqual(1, raw.ReadFloat());
+                Assert.AreEqual(new float[] { 1 }, raw.ReadFloatArray());
+                Assert.AreEqual(1, raw.ReadDouble());
+                Assert.AreEqual(new double[] { 1 }, raw.ReadDoubleArray());
+                Assert.AreEqual(1, raw.ReadDecimal());
+                Assert.AreEqual(new decimal?[] { 1 }, raw.ReadDecimalArray());
+                Assert.AreEqual(Date, raw.ReadTimestamp());
+                Assert.AreEqual(new DateTime?[] { Date }, raw.ReadTimestampArray());
+                Assert.AreEqual("1", raw.ReadString());
+                Assert.AreEqual(new[] { "1" }, raw.ReadStringArray());
+                Assert.AreEqual(Guid, raw.ReadGuid());
+                Assert.AreEqual(new Guid?[] { Guid }, raw.ReadGuidArray());
+                Assert.AreEqual(MyEnum.Bar, raw.ReadEnum<MyEnum>());
+                Assert.AreEqual(new[] { MyEnum.Bar }, raw.ReadEnumArray<MyEnum>());
+            }
+        }
+
+        private enum MyEnum
+        {
+            Bar
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/IO/BinaryStreamsTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/IO/BinaryStreamsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/IO/BinaryStreamsTest.cs
new file mode 100644
index 0000000..ad5358d
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/IO/BinaryStreamsTest.cs
@@ -0,0 +1,151 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Binary.IO
+{
+    using System;
+    using System.IO;
+    using System.Text;
+    using Apache.Ignite.Core.Impl.Binary.IO;
+    using Apache.Ignite.Core.Impl.Memory;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests binary streams.
+    /// </summary>
+    public class BinaryStreamsTest
+    {
+        /// <summary>
+        /// Tests the platform memory stream.
+        /// </summary>
+        [Test]
+        public void TestPlatformMemoryStream()
+        {
+            var stream = new PlatformMemoryStream(GetMemory());
+            TestStream(stream, false, () => stream.SynchronizeOutput());
+        }
+
+        /// <summary>
+        /// Tests the platform big endian memory stream.
+        /// </summary>
+        [Test]
+        public void TestPlatformBigEndianMemoryStream()
+        {
+            var stream = new PlatformBigEndianMemoryStream(GetMemory());
+            TestStream(stream, false, () => stream.SynchronizeOutput());
+        }
+
+        /// <summary>
+        /// Tests the binary heap stream.
+        /// </summary>
+        [Test]
+        public void TestBinaryHeapStream()
+        {
+            TestStream(new BinaryHeapStream(1), true, () => { });
+        }
+
+        /// <summary>
+        /// Gets the memory.
+        /// </summary>
+        private static PlatformMemory GetMemory()
+        {
+            return new PlatformMemoryPool().Allocate(10);
+        }
+
+        /// <summary>
+        /// Tests the stream.
+        /// </summary>
+        private static unsafe void TestStream(IBinaryStream stream, bool sameArr, Action flush)
+        {
+            Action seek = () => Assert.AreEqual(0, stream.Seek(0, SeekOrigin.Begin));
+
+            Action<Action, Func<object>, object> check = (write, read, expectedResult) =>
+            {
+                seek();
+                write();
+                flush();
+                seek();
+                Assert.AreEqual(expectedResult, read());
+            };
+
+            // Arrays.
+            Assert.AreEqual(sameArr, stream.IsSameArray(stream.GetArray()));
+            Assert.IsFalse(stream.IsSameArray(new byte[1]));
+            Assert.IsFalse(stream.IsSameArray(stream.GetArrayCopy()));
+
+            // byte*
+            byte* bytes = stackalloc byte[10];
+            *bytes = 1;
+            *(bytes + 1) = 2;
+
+            stream.Write(bytes, 2);
+            Assert.AreEqual(2, stream.Position);
+            flush();
+
+            seek();
+            Assert.AreEqual(sameArr ? 256 : 2, stream.Remaining);
+            byte* bytes2 = stackalloc byte[2];
+            stream.Read(bytes2, 2);
+            Assert.AreEqual(1, *bytes2);
+            Assert.AreEqual(2, *(bytes2 + 1));
+
+            // char*
+            seek();
+            char* chars = stackalloc char[10];
+            *chars = 'a';
+            *(chars + 1) = 'b';
+
+            Assert.AreEqual(2, stream.WriteString(chars, 2, 2, Encoding.ASCII));
+            flush();
+
+            seek();
+            stream.Read(bytes2, 2);
+            Assert.AreEqual('a', *bytes2);
+            Assert.AreEqual('b', *(bytes2 + 1));
+
+            // Others.
+            check(() => stream.Write(new byte[] {3, 4, 5}, 1, 2), () => stream.ReadByteArray(2), new byte[] {4, 5});
+
+            check(() => stream.WriteBool(true), () => stream.ReadBool(), true);
+            check(() => stream.WriteBoolArray(new[] {true, false}), () => stream.ReadBoolArray(2), 
+                new[] {true, false});
+
+            check(() => stream.WriteByte(4), () => stream.ReadByte(), 4);
+            check(() => stream.WriteByteArray(new byte[] {4, 5, 6}), () => stream.ReadByteArray(3), 
+                new byte[] {4, 5, 6});
+
+            check(() => stream.WriteChar('x'), () => stream.ReadChar(), 'x');
+            check(() => stream.WriteCharArray(new[] {'a', 'b'}), () => stream.ReadCharArray(2), new[] {'a', 'b'});
+
+            check(() => stream.WriteDouble(4), () => stream.ReadDouble(), 4d);
+            check(() => stream.WriteDoubleArray(new[] {4d}), () => stream.ReadDoubleArray(1), new[] {4d});
+
+            check(() => stream.WriteFloat(4), () => stream.ReadFloat(), 4f);
+            check(() => stream.WriteFloatArray(new[] {4f}), () => stream.ReadFloatArray(1), new[] {4f});
+
+            check(() => stream.WriteInt(4), () => stream.ReadInt(), 4);
+            check(() => stream.WriteInt(0, 4), () => stream.ReadInt(), 4);
+            check(() => stream.WriteIntArray(new[] {4}), () => stream.ReadIntArray(1), new[] {4});
+
+            check(() => stream.WriteLong(4), () => stream.ReadLong(), 4L);
+            check(() => stream.WriteLongArray(new[] {4L}), () => stream.ReadLongArray(1), new[] {4L});
+
+            check(() => stream.WriteShort(4), () => stream.ReadShort(), (short)4);
+            check(() => stream.WriteShortArray(new short[] {4}), () => stream.ReadShortArray(1), new short[] {4});
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityKeyTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityKeyTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityKeyTest.cs
new file mode 100644
index 0000000..cf86273
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityKeyTest.cs
@@ -0,0 +1,66 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Cache.Affinity
+{
+    using Apache.Ignite.Core.Cache.Affinity;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests for <see cref="AffinityKey"/>
+    /// </summary>
+    public class AffinityKeyTest
+    {
+        /// <summary>
+        /// Tests the equality.
+        /// </summary>
+        [Test]
+        public void TestEquality()
+        {
+            // Default.
+            var key = new AffinityKey();
+
+            Assert.IsNull(key.Key);
+            Assert.IsNull(key.Affinity);
+            Assert.AreEqual(0, key.GetHashCode());
+            Assert.AreEqual(new AffinityKey(), key);
+
+            // Ctor 1.
+            const string myKey = "myKey";
+            key = new AffinityKey(myKey);
+
+            Assert.AreEqual(myKey, key.Key);
+            Assert.AreEqual(myKey, key.Affinity);
+            Assert.AreNotEqual(0, key.GetHashCode());
+
+            // Ctor 2.
+            var ver1 = new AffinityKey(long.MaxValue, int.MaxValue);
+            var ver2 = new AffinityKey(long.MaxValue, int.MaxValue);
+
+            Assert.AreEqual(ver1, ver2);
+            Assert.IsTrue(ver1 == ver2);
+            Assert.IsFalse(ver1 != ver2);
+
+            Assert.AreNotEqual(key, ver1);
+            Assert.IsTrue(key != ver1);
+            Assert.IsFalse(key == ver1);
+
+            // ToString.
+            Assert.AreEqual("AffinityKey [Key=1, Affinity=2]", new AffinityKey(1, 2).ToString());
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityTopologyVersionTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityTopologyVersionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityTopologyVersionTest.cs
new file mode 100644
index 0000000..de25ea4
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Affinity/AffinityTopologyVersionTest.cs
@@ -0,0 +1,59 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Cache.Affinity
+{
+    using Apache.Ignite.Core.Cache.Affinity;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests for <see cref="AffinityTopologyVersion"/>
+    /// </summary>
+    public class AffinityTopologyVersionTest
+    {
+        /// <summary>
+        /// Tests the equality.
+        /// </summary>
+        [Test]
+        public void TestEquality()
+        {
+            // Default.
+            var ver = new AffinityTopologyVersion();
+
+            Assert.AreEqual(0, ver.Version);
+            Assert.AreEqual(0, ver.MinorVersion);
+            Assert.AreEqual(0, ver.GetHashCode());
+            Assert.AreEqual(new AffinityTopologyVersion(), ver);
+
+            // Custom.
+            var ver1 = new AffinityTopologyVersion(long.MaxValue, int.MaxValue);
+            var ver2 = new AffinityTopologyVersion(long.MaxValue, int.MaxValue);
+
+            Assert.AreEqual(ver1, ver2);
+            Assert.IsTrue(ver1 == ver2);
+            Assert.IsFalse(ver1 != ver2);
+
+            Assert.AreNotEqual(ver, ver1);
+            Assert.IsTrue(ver != ver1);
+            Assert.IsFalse(ver == ver1);
+
+            // ToString.
+            Assert.AreEqual("AffinityTopologyVersion [Version=1, MinorVersion=2]", 
+                new AffinityTopologyVersion(1, 2).ToString());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
index 8878d2e..9fd1f1d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
@@ -29,7 +29,6 @@ namespace Apache.Ignite.Core.Tests.Cache
     using Apache.Ignite.Core.Cache.Expiry;
     using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Common;
-    using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Impl.Cache;
     using Apache.Ignite.Core.Tests.Query;
     using Apache.Ignite.Core.Transactions;
@@ -991,10 +990,8 @@ namespace Apache.Ignite.Core.Tests.Cache
                 key1 = PrimaryKeyForCache(Cache(1));
             }
 
-            var cache = cache0.WithExpiryPolicy(new ExpiryPolicy(null, null, null));
-
             // Test zero expiration.
-            cache = cache0.WithExpiryPolicy(new ExpiryPolicy(TimeSpan.Zero, TimeSpan.Zero, TimeSpan.Zero));
+            var cache = cache0.WithExpiryPolicy(new ExpiryPolicy(TimeSpan.Zero, TimeSpan.Zero, TimeSpan.Zero));
 
             cache.Put(key0, key0);
             cache.Put(key1, key1);
@@ -1930,7 +1927,7 @@ namespace Apache.Ignite.Core.Tests.Cache
             };
 
             checkLock(cache.Lock(key));
-            checkLock(cache.LockAll(new[] { key, 1, 2, 3 }));
+            checkLock(cache.LockAll(new[] {key, 1, 2, 3}));
         }
 
         [Test]
@@ -2038,7 +2035,7 @@ namespace Apache.Ignite.Core.Tests.Cache
         }
 
         /// <summary>
-        /// ENsure taht lock cannot be obtained by other threads.
+        /// Ensure that lock cannot be obtained by other threads.
         /// </summary>
         /// <param name="getLock">Get lock function.</param>
         /// <param name="sharedLock">Shared lock.</param>
@@ -2419,10 +2416,15 @@ namespace Apache.Ignite.Core.Tests.Cache
                 return;
 
             var tx = Transactions.TxStart();
-            
+
             Assert.AreEqual(TransactionState.Active, tx.State);
+            Assert.AreEqual(Thread.CurrentThread.ManagedThreadId, tx.ThreadId);
 
-            tx.Rollback();
+            tx.AddMeta("myMeta", 42);
+            Assert.AreEqual(42, tx.Meta<int>("myMeta"));
+            Assert.AreEqual(42, tx.RemoveMeta<int>("myMeta"));
+
+            tx.RollbackAsync().Wait();
 
             Assert.AreEqual(TransactionState.RolledBack, tx.State);
 
@@ -3346,7 +3348,7 @@ namespace Apache.Ignite.Core.Tests.Cache
             return true;
         }
 
-        protected virtual bool LockingEnabled()
+        protected bool LockingEnabled()
         {
             return TxEnabled();
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheResultTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheResultTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheResultTest.cs
new file mode 100644
index 0000000..6be26bf
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheResultTest.cs
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Cache
+{
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Cache;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// <see cref="CacheResult{T}"/> tests.
+    /// </summary>
+    public class CacheResultTest
+    {
+        /// <summary>
+        /// Tests equality members.
+        /// </summary>
+        [Test]
+        public void TestEquality()
+        {
+            var entry1 = new CacheResult<int>(2);
+            var entry2 = new CacheResult<int>(2);
+            var entry3 = new CacheResult<int>(3);
+
+            Assert.AreEqual(entry1, entry2);
+            Assert.AreNotEqual(entry1, entry3);
+
+            Assert.IsTrue(entry1 == entry2);
+            Assert.IsFalse(entry1 != entry2);
+
+            Assert.IsTrue(entry1 != entry3);
+            Assert.IsFalse(entry1 == entry3);
+
+            var boxedEntry1 = (object) entry1;
+            var boxedEntry2 = (object) entry2;
+            var boxedEntry3 = (object) entry3;
+
+            Assert.IsFalse(ReferenceEquals(boxedEntry1, boxedEntry2));
+
+            Assert.AreEqual(boxedEntry1, boxedEntry2);
+            Assert.AreNotEqual(boxedEntry1, boxedEntry3);
+        }
+
+        /// <summary>
+        /// Tests with hash data structures.
+        /// </summary>
+        [Test]
+        public void TestHashCode()
+        {
+            var entry1 = new CacheResult<int>(2);
+            var entry2 = new CacheResult<int>(2);
+            var entry3 = new CacheResult<int>(3);
+
+            var set = new HashSet<object> {entry1};
+
+            Assert.IsTrue(set.Contains(entry1));
+            Assert.IsTrue(set.Contains(entry2));
+            Assert.IsFalse(set.Contains(entry3));
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAbstractTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAbstractTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAbstractTest.cs
index 4b285f9..270c3fc 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAbstractTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Continuous/ContinuousQueryAbstractTest.cs
@@ -181,11 +181,11 @@ namespace Apache.Ignite.Core.Tests.Cache.Query.Continuous
             {
                 // Put from local node.
                 cache1.GetAndPut(key1, Entry(key1));
-                CheckCallbackSingle(key1, null, Entry(key1));
+                CheckCallbackSingle(key1, null, Entry(key1), CacheEntryEventType.Created);
 
                 // Put from remote node.
                 cache2.GetAndPut(key2, Entry(key2));
-                CheckCallbackSingle(key2, null, Entry(key2));
+                CheckCallbackSingle(key2, null, Entry(key2), CacheEntryEventType.Created);
             }
 
             qryHnd.Dispose();
@@ -217,13 +217,13 @@ namespace Apache.Ignite.Core.Tests.Cache.Query.Continuous
             {
                 // Put from local node.
                 cache1.GetAndPut(key1, Entry(key1));
-                CheckCallbackSingle(key1, null, Entry(key1));
+                CheckCallbackSingle(key1, null, Entry(key1), CacheEntryEventType.Created);
 
                 cache1.GetAndPut(key1, Entry(key1 + 1));
-                CheckCallbackSingle(key1, Entry(key1), Entry(key1 + 1));
+                CheckCallbackSingle(key1, Entry(key1), Entry(key1 + 1), CacheEntryEventType.Updated);
 
                 cache1.Remove(key1);
-                CheckCallbackSingle(key1, Entry(key1 + 1), null);
+                CheckCallbackSingle(key1, Entry(key1 + 1), null, CacheEntryEventType.Removed);
 
                 // Put from remote node.
                 cache2.GetAndPut(key2, Entry(key2));
@@ -231,21 +231,21 @@ namespace Apache.Ignite.Core.Tests.Cache.Query.Continuous
                 if (loc)
                     CheckNoCallback(100);
                 else
-                    CheckCallbackSingle(key2, null, Entry(key2));
+                    CheckCallbackSingle(key2, null, Entry(key2), CacheEntryEventType.Created);
 
                 cache1.GetAndPut(key2, Entry(key2 + 1));
 
                 if (loc)
                     CheckNoCallback(100);
                 else
-                    CheckCallbackSingle(key2, Entry(key2), Entry(key2 + 1));
+                    CheckCallbackSingle(key2, Entry(key2), Entry(key2 + 1), CacheEntryEventType.Updated);
 
                 cache1.Remove(key2);
 
                 if (loc)
                     CheckNoCallback(100);
                 else
-                    CheckCallbackSingle(key2, Entry(key2 + 1), null);
+                    CheckCallbackSingle(key2, Entry(key2 + 1), null, CacheEntryEventType.Removed);
             }
 
             cache1.Put(key1, Entry(key1));
@@ -310,7 +310,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query.Continuous
                 int key1 = PrimaryKey(cache1);
                 cache1.GetAndPut(key1, Entry(key1));
                 CheckFilterSingle(key1, null, Entry(key1));
-                CheckCallbackSingle(key1, null, Entry(key1));
+                CheckCallbackSingle(key1, null, Entry(key1), CacheEntryEventType.Created);
 
                 // Put from remote node.
                 int key2 = PrimaryKey(cache2);
@@ -324,7 +324,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query.Continuous
                 else
                 {
                     CheckFilterSingle(key2, null, Entry(key2));
-                    CheckCallbackSingle(key2, null, Entry(key2));
+                    CheckCallbackSingle(key2, null, Entry(key2), CacheEntryEventType.Created);
                 }
 
                 AbstractFilter<BinarizableEntry>.res = false;
@@ -755,12 +755,12 @@ namespace Apache.Ignite.Core.Tests.Cache.Query.Continuous
             {
                 // Put from local node.
                 cache1.GetAndPut(key1, Entry(key1));
-                CheckCallbackSingle(key1, null, Entry(key1));
+                CheckCallbackSingle(key1, null, Entry(key1), CacheEntryEventType.Created);
 
                 // Put from remote node.
                 cache1.GetAndPut(key2, Entry(key2));
                 CheckNoCallback(100);
-                CheckCallbackSingle(key2, null, Entry(key2), 1000);
+                CheckCallbackSingle(key2, null, Entry(key2), CacheEntryEventType.Created);
             }
         }
 
@@ -851,7 +851,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query.Continuous
 
                     // Check continuous query
                     cache1.Put(44, Entry(44));
-                    CheckCallbackSingle(44, null, Entry(44));
+                    CheckCallbackSingle(44, null, Entry(44), CacheEntryEventType.Created);
                 }
 
                 Assert.Throws<ObjectDisposedException>(() => contQry.GetInitialQueryCursor());
@@ -922,19 +922,10 @@ namespace Apache.Ignite.Core.Tests.Cache.Query.Continuous
         /// <param name="expKey">Expected key.</param>
         /// <param name="expOldVal">Expected old value.</param>
         /// <param name="expVal">Expected new value.</param>
-        private static void CheckCallbackSingle(int expKey, BinarizableEntry expOldVal, BinarizableEntry expVal)
-        {
-            CheckCallbackSingle(expKey, expOldVal, expVal, 1000);
-        }
-
-        /// <summary>
-        /// Check single callback event.
-        /// </summary>
-        /// <param name="expKey">Expected key.</param>
-        /// <param name="expOldVal">Expected old value.</param>
-        /// <param name="expVal">Expected new value.</param>
+        /// <param name="expType">Expected type.</param>
         /// <param name="timeout">Timeout.</param>
-        private static void CheckCallbackSingle(int expKey, BinarizableEntry expOldVal, BinarizableEntry expVal, int timeout)
+        private static void CheckCallbackSingle(int expKey, BinarizableEntry expOldVal, BinarizableEntry expVal,
+            CacheEntryEventType expType, int timeout = 1000)
         {
             CallbackEvent evt;
 
@@ -946,6 +937,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query.Continuous
             Assert.AreEqual(expKey, e.Key);
             Assert.AreEqual(expOldVal, e.OldValue);
             Assert.AreEqual(expVal, e.Value);
+            Assert.AreEqual(expType, e.EventType);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheParallelLoadStoreTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheParallelLoadStoreTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheParallelLoadStoreTest.cs
index 4aa910c..105dea2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheParallelLoadStoreTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheParallelLoadStoreTest.cs
@@ -96,7 +96,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Store
             }
 
             // check that items were processed in parallel
-            Assert.GreaterOrEqual(CacheTestParallelLoadStore.UniqueThreadCount, Environment.ProcessorCount);
+            Assert.GreaterOrEqual(CacheTestParallelLoadStore.UniqueThreadCount, Environment.ProcessorCount - 1);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreAdapterTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreAdapterTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreAdapterTest.cs
new file mode 100644
index 0000000..6690584
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreAdapterTest.cs
@@ -0,0 +1,90 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Cache.Store
+{
+    using System.Collections.Generic;
+    using System.Linq;
+    using Apache.Ignite.Core.Cache.Store;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests for <see cref="CacheStoreAdapter"/>.
+    /// </summary>
+    public class CacheStoreAdapterTest
+    {
+        /// <summary>
+        /// Tests the load write delete.
+        /// </summary>
+        [Test]
+        public void TestLoadWriteDelete()
+        {
+            var store = new Store();
+
+            store.LoadCache(null);
+            Assert.IsEmpty(store.Map);
+
+            var data = Enumerable.Range(1, 5).ToDictionary(x => x, x => x.ToString());
+
+            // Write.
+            store.WriteAll(data);
+            Assert.AreEqual(data, store.Map);
+
+            // Load.
+            CollectionAssert.AreEqual(data, store.LoadAll(data.Keys));
+            CollectionAssert.AreEqual(data.Where(x => x.Key < 3).ToDictionary(x => x.Key, x => x.Value),
+                store.LoadAll(data.Keys.Where(x => x < 3).ToList()));
+
+            // Delete.
+            var removed = new[] {3, 5};
+
+            foreach (var key in removed)
+                data.Remove(key);
+
+            store.DeleteAll(removed);
+            CollectionAssert.AreEqual(data, store.LoadAll(data.Keys));
+        }
+
+        /// <summary>
+        /// Test store.
+        /// </summary>
+        private class Store : CacheStoreAdapter
+        {
+            /** */
+            public readonly Dictionary<object, object> Map = new Dictionary<object, object>();
+
+            /** <inheritdoc /> */
+            public override object Load(object key)
+            {
+                object res;
+                return Map.TryGetValue(key, out res) ? res : null;
+            }
+
+            /** <inheritdoc /> */
+            public override void Write(object key, object val)
+            {
+                Map[key] = val;
+            }
+
+            /** <inheritdoc /> */
+            public override void Delete(object key)
+            {
+                Map.Remove(key);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestParallelLoadStore.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestParallelLoadStore.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestParallelLoadStore.cs
index 770ca83..81b4697 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestParallelLoadStore.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheTestParallelLoadStore.cs
@@ -17,6 +17,7 @@
 
 namespace Apache.Ignite.Core.Tests.Cache.Store
 {
+    using System;
     using System.Collections;
     using System.Collections.Concurrent;
     using System.Collections.Generic;
@@ -36,6 +37,14 @@ namespace Apache.Ignite.Core.Tests.Cache.Store
         private static readonly ConcurrentDictionary<int, int> ThreadIds = new ConcurrentDictionary<int, int>();
 
         /// <summary>
+        /// Initializes a new instance of the <see cref="CacheTestParallelLoadStore"/> class.
+        /// </summary>
+        public CacheTestParallelLoadStore()
+        {
+            MaxDegreeOfParallelism -= 1;
+        }
+
+        /// <summary>
         /// Gets the count of unique threads that entered Parse method.
         /// </summary>
         public static int UniqueThreadCount

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/MultiValueDictionaryTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/MultiValueDictionaryTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/MultiValueDictionaryTest.cs
new file mode 100644
index 0000000..aa3e2aa
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/MultiValueDictionaryTest.cs
@@ -0,0 +1,58 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Collections
+{
+    using Apache.Ignite.Core.Impl.Collections;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests the <see cref="MultiValueDictionary{TKey,TValue}"/>.
+    /// </summary>
+    public class MultiValueDictionaryTest
+    {
+        /// <summary>
+        /// Tests the dictionary.
+        /// </summary>
+        [Test]
+        public void TestMultiValueDictionary()
+        {
+            var dict = new MultiValueDictionary<int, int>();
+
+            dict.Add(1, 1);
+            dict.Add(1, 2);
+
+            int val;
+
+            Assert.IsTrue(dict.TryRemove(1, out val));
+            Assert.AreEqual(2, val);
+
+            Assert.IsTrue(dict.TryRemove(1, out val));
+            Assert.AreEqual(1, val);
+
+            Assert.IsFalse(dict.TryRemove(1, out val));
+
+            dict.Add(2, 1);
+            dict.Add(2, 2);
+            dict.Remove(2, 3);
+            dict.Remove(2, 2);
+
+            Assert.IsTrue(dict.TryRemove(2, out val));
+            Assert.AreEqual(1, val);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/ReadOnlyCollectionTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/ReadOnlyCollectionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/ReadOnlyCollectionTest.cs
new file mode 100644
index 0000000..27991a3
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/ReadOnlyCollectionTest.cs
@@ -0,0 +1,59 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Collections
+{
+    using System;
+    using System.Collections;
+    using System.Linq;
+    using Apache.Ignite.Core.Impl.Collections;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests for <see cref="ReadOnlyCollection{T}"/>
+    /// </summary>
+    public class ReadOnlyCollectionTest
+    {
+        /// <summary>
+        /// Tests the disctionary.
+        /// </summary>
+        [Test]
+        public void TestCollection()
+        {
+            // Default ctor.
+            var data = Enumerable.Range(1, 5).ToArray();
+            var col = new ReadOnlyCollection<int>(data);
+
+            Assert.AreEqual(5, col.Count);
+            Assert.IsTrue(col.IsReadOnly);
+            CollectionAssert.AreEqual(data, col);
+
+            Assert.IsTrue(col.GetEnumerator().MoveNext());
+            Assert.IsTrue(((IEnumerable) col).GetEnumerator().MoveNext());
+
+            Assert.IsTrue(col.Contains(4));
+
+            var arr = new int[5];
+            col.CopyTo(arr, 0);
+            CollectionAssert.AreEqual(data, arr);
+
+            Assert.Throws<NotSupportedException>(() => col.Add(1));
+            Assert.Throws<NotSupportedException>(() => col.Clear());
+            Assert.Throws<NotSupportedException>(() => col.Remove(1));
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/ReadOnlyDictionaryTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/ReadOnlyDictionaryTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/ReadOnlyDictionaryTest.cs
new file mode 100644
index 0000000..294251a
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Collections/ReadOnlyDictionaryTest.cs
@@ -0,0 +1,70 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Collections
+{
+    using System;
+    using System.Collections;
+    using System.Collections.Generic;
+    using System.Linq;
+    using Apache.Ignite.Core.Impl.Collections;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests for <see cref="ReadOnlyDictionary{TKey,TValue}"/>
+    /// </summary>
+    public class ReadOnlyDictionaryTest
+    {
+        /// <summary>
+        /// Tests the disctionary.
+        /// </summary>
+        [Test]
+        public void TestDictionary()
+        {
+            // Default ctor.
+            var data = Enumerable.Range(1, 5).ToDictionary(x => x, x => x.ToString());
+            var dict = new ReadOnlyDictionary<int, string>(data);
+
+            Assert.AreEqual(5, dict.Count);
+            Assert.IsTrue(dict.IsReadOnly);
+            CollectionAssert.AreEqual(data, dict);
+            CollectionAssert.AreEqual(data.Keys, dict.Keys);
+            CollectionAssert.AreEqual(data.Values, dict.Values);
+
+            Assert.IsTrue(dict.GetEnumerator().MoveNext());
+            Assert.IsTrue(((IEnumerable) dict).GetEnumerator().MoveNext());
+
+            Assert.IsTrue(dict.ContainsKey(1));
+            Assert.IsTrue(dict.Contains(new KeyValuePair<int, string>(4, "4")));
+            Assert.AreEqual("3", dict[3]);
+
+            string val;
+            Assert.IsTrue(dict.TryGetValue(2, out val));
+            Assert.AreEqual("2", val);
+
+            var arr = new KeyValuePair<int, string>[5];
+            dict.CopyTo(arr, 0);
+            CollectionAssert.AreEqual(data, arr);
+
+            Assert.Throws<NotSupportedException>(() => dict.Add(1, "2"));
+            Assert.Throws<NotSupportedException>(() => dict.Add(new KeyValuePair<int, string>(1, "2")));
+            Assert.Throws<NotSupportedException>(() => dict.Clear());
+            Assert.Throws<NotSupportedException>(() => dict.Remove(1));
+            Assert.Throws<NotSupportedException>(() => dict.Remove(new KeyValuePair<int, string>(1, "2")));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Common/IgniteGuidTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Common/IgniteGuidTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Common/IgniteGuidTest.cs
new file mode 100644
index 0000000..705faf4
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Common/IgniteGuidTest.cs
@@ -0,0 +1,62 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Common
+{
+    using System;
+    using Apache.Ignite.Core.Common;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests the <see cref="IgniteGuid"/>.
+    /// </summary>
+    public class IgniteGuidTest
+    {
+        /// <summary>
+        /// Tests the <see cref="IgniteGuid"/>.
+        /// </summary>
+        [Test]
+        public void TestIgniteGuid()
+        {
+            var guid = Guid.NewGuid();
+
+            var id1 = new IgniteGuid(guid, 1);
+            var id2 = new IgniteGuid(guid, 1);
+            var id3 = new IgniteGuid(guid, 2);
+            var id4 = new IgniteGuid(Guid.NewGuid(), 2);
+
+            // Properties.
+            Assert.AreEqual(guid, id1.GlobalId);
+            Assert.AreEqual(1, id1.LocalId);
+            Assert.AreEqual(id1.GetHashCode(), id2.GetHashCode());
+
+            // Equality.
+            Assert.AreEqual(id1, id2);
+            Assert.IsTrue(id1 == id2);
+            Assert.IsFalse(id1 != id2);
+
+            // Inequality.
+            Assert.AreNotEqual(id1, id3);
+            Assert.IsFalse(id1 == id3);
+            Assert.IsTrue(id1 != id3);
+
+            Assert.AreNotEqual(id4, id3);
+            Assert.IsFalse(id4 == id3);
+            Assert.IsTrue(id4 != id3);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/AbstractTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/AbstractTaskTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/AbstractTaskTest.cs
index d31ad43..6bcd010 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/AbstractTaskTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/AbstractTaskTest.cs
@@ -162,30 +162,21 @@ namespace Apache.Ignite.Core.Tests.Compute
         /// </summary>
         /// <param name="path">Path to Java XML configuration.</param>
         /// <returns>Node configuration.</returns>
-        protected IgniteConfiguration Configuration(string path)
+        private IgniteConfiguration Configuration(string path)
         {
-            IgniteConfiguration cfg = new IgniteConfiguration();
-
-            if (!_fork)
+            return new IgniteConfiguration
             {
-                BinaryConfiguration portCfg = new BinaryConfiguration();
-
-                ICollection<BinaryTypeConfiguration> portTypeCfgs = new List<BinaryTypeConfiguration>();
-
-                GetBinaryTypeConfigurations(portTypeCfgs);
-
-                portCfg.TypeConfigurations = portTypeCfgs;
-
-                cfg.BinaryConfiguration = portCfg;
-            }
-
-            cfg.JvmClasspath = TestUtils.CreateTestClasspath();
-
-            cfg.JvmOptions = TestUtils.TestJavaOptions();
-
-            cfg.SpringConfigUrl = path;
-
-            return cfg;
+                JvmClasspath = TestUtils.CreateTestClasspath(),
+                JvmOptions = TestUtils.TestJavaOptions(),
+                SpringConfigUrl = path,
+                BinaryConfiguration = _fork
+                    ? null
+                    : new BinaryConfiguration
+                    {
+                        TypeConfigurations =
+                            (GetBinaryTypes() ?? new Type[0]).Select(t => new BinaryTypeConfiguration(t)).ToList()
+                    }
+            };
         }
 
         /// <summary>
@@ -209,10 +200,9 @@ namespace Apache.Ignite.Core.Tests.Compute
         /// <summary>
         /// Define binary types.
         /// </summary>
-        /// <param name="portTypeCfgs">Binary type configurations.</param>
-        protected virtual void GetBinaryTypeConfigurations(ICollection<BinaryTypeConfiguration> portTypeCfgs)
+        protected virtual ICollection<Type> GetBinaryTypes()
         {
-            // No-op.
+            return null;
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableClosureTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableClosureTaskTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableClosureTaskTest.cs
index b881582..c169f1e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableClosureTaskTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableClosureTaskTest.cs
@@ -40,12 +40,15 @@ namespace Apache.Ignite.Core.Tests.Compute
         protected BinarizableClosureTaskTest(bool fork) : base(fork) { }
 
         /** <inheritDoc /> */
-        protected override void GetBinaryTypeConfigurations(ICollection<BinaryTypeConfiguration> portTypeCfgs)
+        protected override ICollection<Type> GetBinaryTypes()
         {
-            portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableOutFunc)));
-            portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableFunc)));
-            portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableResult)));
-            portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableException)));
+            return new []
+            {
+                typeof(BinarizableOutFunc),
+                typeof(BinarizableFunc),
+                typeof(BinarizableResult),
+                typeof(BinarizableException)
+            };
         }
 
         /** <inheritDoc /> */
@@ -76,6 +79,11 @@ namespace Apache.Ignite.Core.Tests.Compute
         {
             Assert.IsTrue(err != null);
 
+            var aggregate = err as AggregateException;
+
+            if (aggregate != null)
+                err = aggregate.InnerException;
+
             BinarizableException err0 = err as BinarizableException;
 
             Assert.IsTrue(err0 != null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableTaskTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableTaskTest.cs
index 8aa28de..6bdfd9c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableTaskTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/BinarizableTaskTest.cs
@@ -17,6 +17,7 @@
 
 namespace Apache.Ignite.Core.Tests.Compute
 {
+    using System;
     using System.Collections.Generic;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cluster;
@@ -71,14 +72,17 @@ namespace Apache.Ignite.Core.Tests.Compute
         }
 
         /** <inheritDoc /> */
-        override protected void GetBinaryTypeConfigurations(ICollection<BinaryTypeConfiguration> portTypeCfgs)
+        protected override ICollection<Type> GetBinaryTypes()
         {
-            portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableJobArgument)));
-            portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableJobResult)));
-            portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableTaskArgument)));
-            portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableTaskResult)));
-            portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableJob)));
-            portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(BinarizableWrapper)));
+            return new[]
+            {
+                typeof(BinarizableJobResult),
+                typeof(BinarizableTaskArgument),
+                typeof(BinarizableTaskResult),
+                typeof(BinarizableJobArgument),
+                typeof(BinarizableJob),
+                typeof(BinarizableWrapper)
+            };
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/CancellationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/CancellationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/CancellationTest.cs
index bbd1169..19bb40d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/CancellationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/CancellationTest.cs
@@ -68,11 +68,21 @@ namespace Apache.Ignite.Core.Tests.Compute
         public void TestClosures()
         {
             TestClosure((c, t) => c.BroadcastAsync(new ComputeAction(), t));
+            TestClosure((c, t) => c.BroadcastAsync(new ComputeFunc(), t));
+            TestClosure((c, t) => c.BroadcastAsync(new ComputeBiFunc(), 10, t));
+
             TestClosure((c, t) => c.AffinityRunAsync(null, 0, new ComputeAction(), t));
+
             TestClosure((c, t) => c.RunAsync(new ComputeAction(), t));
             TestClosure((c, t) => c.RunAsync(Enumerable.Range(1, 10).Select(x => new ComputeAction()), t));
+
             TestClosure((c, t) => c.CallAsync(new ComputeFunc(), t));
+            TestClosure((c, t) => c.CallAsync(Enumerable.Range(1, 10).Select(x => new ComputeFunc()), t));
+            TestClosure((c, t) => c.CallAsync(Enumerable.Range(1, 10).Select(x => new ComputeFunc()), 
+                new ComputeReducer(), t));
+
             TestClosure((c, t) => c.AffinityCallAsync(null, 0, new ComputeFunc(), t));
+
             TestClosure((c, t) => c.ApplyAsync(new ComputeBiFunc(), 10, t));
             TestClosure((c, t) => c.ApplyAsync(new ComputeBiFunc(), Enumerable.Range(1, 100), t));
             TestClosure((c, t) => c.ApplyAsync(new ComputeBiFunc(), Enumerable.Range(1, 100), new ComputeReducer(), t));

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ClosureTaskTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ClosureTaskTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ClosureTaskTest.cs
index 8664413..ffb2844 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ClosureTaskTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ClosureTaskTest.cs
@@ -19,6 +19,7 @@ namespace Apache.Ignite.Core.Tests.Compute
 {
     using System;
     using System.Collections.Generic;
+    using System.Linq;
     using Apache.Ignite.Core.Compute;
     using NUnit.Framework;
 
@@ -27,7 +28,7 @@ namespace Apache.Ignite.Core.Tests.Compute
     /// </summary>
     public abstract class ClosureTaskTest : AbstractTaskTest
     {
-        /** Amount of multiple clousres. */
+        /** Amount of multiple closures. */
         private const int MultiCloCnt = 5;
 
         /** */
@@ -45,9 +46,8 @@ namespace Apache.Ignite.Core.Tests.Compute
         [Test]
         public void TestExecuteSingle()
         {
-            var res = Grid1.GetCompute().Call(OutFunc(false));
-
-            CheckResult(res);
+            CheckResult(Grid1.GetCompute().Call(OutFunc(false)));
+            CheckResult(Grid1.GetCompute().CallAsync(OutFunc(false)).Result);
         }
 
         /// <summary>
@@ -56,16 +56,8 @@ namespace Apache.Ignite.Core.Tests.Compute
         [Test]
         public void TestExecuteSingleException()
         {
-            try
-            {
-                Grid1.GetCompute().Call(OutFunc(true));
-
-                Assert.Fail();
-            }
-            catch (Exception e)
-            {
-                CheckError(e);
-            }
+            CheckError(Assert.Catch(() => Grid1.GetCompute().Call(OutFunc(true))));
+            CheckError(Assert.Catch(() => Grid1.GetCompute().CallAsync(OutFunc(true)).Wait()));
         }
 
         /// <summary>
@@ -74,15 +66,10 @@ namespace Apache.Ignite.Core.Tests.Compute
         [Test]
         public void TestExecuteMultiple()
         {
-            var clos = new List<IComputeFunc<object>>(MultiCloCnt);
-
-            for (int i = 0; i < MultiCloCnt; i++)
-                clos.Add(OutFunc(false));
-
-            ICollection<object> ress = Grid1.GetCompute().Call(clos);
+            var clos = Enumerable.Range(0, MultiCloCnt).Select(x => OutFunc(false)).ToArray();
 
-            foreach (object res in ress)
-                CheckResult(res);
+            Grid1.GetCompute().Call(clos).ToList().ForEach(CheckResult);
+            Grid1.GetCompute().CallAsync(clos).Result.ToList().ForEach(CheckResult);
         }
 
         /// <summary>
@@ -91,15 +78,10 @@ namespace Apache.Ignite.Core.Tests.Compute
         [Test]
         public void TestExecuteMultipleReduced()
         {
-            var clos = new List<IComputeFunc<object>>(MultiCloCnt);
+            var clos = Enumerable.Range(0, MultiCloCnt).Select(x => OutFunc(false)).ToArray();
 
-            for (int i = 0; i < MultiCloCnt; i++)
-                clos.Add(OutFunc(false));
-
-            ICollection<object> ress = Grid1.GetCompute().Call(clos, new Reducer(false));
-
-            foreach (object res in ress)
-                CheckResult(res);
+            Grid1.GetCompute().Call(clos, new Reducer(false)).ToList().ForEach(CheckResult);
+            Grid1.GetCompute().CallAsync(clos, new Reducer(false)).Result.ToList().ForEach(CheckResult);
         }
 
         /// <summary>
@@ -108,21 +90,11 @@ namespace Apache.Ignite.Core.Tests.Compute
         [Test]
         public void TestExecuteMultipleException()
         {
-            var clos = new List<IComputeFunc<object>>(MultiCloCnt);
+            // Some closures will be faulty.
+            var clos = Enumerable.Range(0, MultiCloCnt).Select(x => OutFunc(x % 2 == 0)).ToArray();
 
-            for (int i = 0; i < MultiCloCnt; i++)
-                clos.Add(OutFunc(i % 2 == 0)); // Some closures will be faulty.
-
-            try
-            {
-                Grid1.GetCompute().Call(clos);
-
-                Assert.Fail();
-            }
-            catch (Exception e)
-            {
-                CheckError(e);
-            }
+            CheckError(Assert.Catch(() => Grid1.GetCompute().Call(clos)));
+            CheckError(Assert.Catch(() => Grid1.GetCompute().CallAsync(clos).Wait()));
         }
 
         /// <summary>
@@ -131,10 +103,8 @@ namespace Apache.Ignite.Core.Tests.Compute
         [Test]
         public void TestBroadcastOut()
         {
-            ICollection<object> ress = Grid1.GetCompute().Broadcast(OutFunc(false));
-
-            foreach (object res in ress)
-                CheckResult(res);
+            Grid1.GetCompute().Broadcast(OutFunc(false)).ToList().ForEach(CheckResult);
+            Grid1.GetCompute().BroadcastAsync(OutFunc(false)).Result.ToList().ForEach(CheckResult);
         }
 
         /// <summary>
@@ -143,16 +113,8 @@ namespace Apache.Ignite.Core.Tests.Compute
         [Test]
         public void TestBroadcastOutException()
         {
-            try
-            {
-                Grid1.GetCompute().Broadcast(OutFunc(true));
-
-                Assert.Fail();
-            }
-            catch (Exception e)
-            {
-                CheckError(e);
-            }
+            CheckError(Assert.Catch(() => Grid1.GetCompute().Broadcast(OutFunc(true))));
+            CheckError(Assert.Catch(() => Grid1.GetCompute().BroadcastAsync(OutFunc(true)).Wait()));
         }
 
         /// <summary>
@@ -161,10 +123,8 @@ namespace Apache.Ignite.Core.Tests.Compute
         [Test]
         public void TestBroadcastInOut()
         {
-            ICollection<object> ress = Grid1.GetCompute().Broadcast(Func(false), 1);
-
-            foreach (object res in ress)
-                CheckResult(res);
+            Grid1.GetCompute().Broadcast(Func(false), 1).ToList().ForEach(CheckResult);
+            Grid1.GetCompute().BroadcastAsync(Func(false), 1).Result.ToList().ForEach(CheckResult);
         }
 
         /// <summary>
@@ -173,16 +133,8 @@ namespace Apache.Ignite.Core.Tests.Compute
         [Test]
         public void TestBroadcastInOutException()
         {
-            try
-            {
-                Grid1.GetCompute().Broadcast(Func(true), 1);
-
-                Assert.Fail();
-            }
-            catch (Exception e)
-            {
-                CheckError(e);
-            }
+            CheckError(Assert.Catch(() => Grid1.GetCompute().Broadcast(Func(true), 1)));
+            CheckError(Assert.Catch(() => Grid1.GetCompute().BroadcastAsync(Func(true), 1).Wait()));
         }
 
         /// <summary>
@@ -191,9 +143,8 @@ namespace Apache.Ignite.Core.Tests.Compute
         [Test]
         public void TestApply()
         {
-            object res = Grid1.GetCompute().Apply(Func(false), 1);
-
-            CheckResult(res);
+            CheckResult(Grid1.GetCompute().Apply(Func(false), 1));
+            CheckResult(Grid1.GetCompute().ApplyAsync(Func(false), 1).Result);
         }
 
         /// <summary>
@@ -202,16 +153,8 @@ namespace Apache.Ignite.Core.Tests.Compute
         [Test]
         public void TestApplyException()
         {
-            try
-            {
-                Grid1.GetCompute().Apply(Func(true), 1);
-
-                Assert.Fail();
-            }
-            catch (Exception e)
-            {
-                CheckError(e);
-            }
+            CheckError(Assert.Catch(() => Grid1.GetCompute().Apply(Func(true), 1)));
+            CheckError(Assert.Catch(() => Grid1.GetCompute().ApplyAsync(Func(true), 1).Wait()));
         }
 
         /// <summary>
@@ -220,19 +163,10 @@ namespace Apache.Ignite.Core.Tests.Compute
         [Test]
         public void TestApplyMultiple()
         {
-            var args = new List<object>(MultiCloCnt);
-
-            for (int i = 0; i < MultiCloCnt; i++)
-                args.Add(1);
-
-            Console.WriteLine("START TASK");
-
-            var ress = Grid1.GetCompute().Apply(Func(false), args);
+            var args = Enumerable.Repeat(1, MultiCloCnt).Cast<object>().ToArray();
 
-            Console.WriteLine("END TASK.");
-
-            foreach (object res in ress)
-                CheckResult(res);
+            Grid1.GetCompute().Apply(Func(false), args).ToList().ForEach(CheckResult);
+            Grid1.GetCompute().ApplyAsync(Func(false), args).Result.ToList().ForEach(CheckResult);
         }
 
         /// <summary>
@@ -241,21 +175,10 @@ namespace Apache.Ignite.Core.Tests.Compute
         [Test]
         public void TestApplyMultipleException()
         {
-            ICollection<int> args = new List<int>(MultiCloCnt);
-
-            for (int i = 0; i < MultiCloCnt; i++)
-                args.Add(1);
-
-            try
-            {
-                Grid1.GetCompute().Apply(Func(true), args);
+            var args = Enumerable.Repeat(1, MultiCloCnt).Cast<object>().ToArray();
 
-                Assert.Fail();
-            }
-            catch (Exception e)
-            {
-                CheckError(e);
-            }
+            CheckError(Assert.Catch(() => Grid1.GetCompute().Apply(Func(true), args)));
+            CheckError(Assert.Catch(() => Grid1.GetCompute().ApplyAsync(Func(true), args).Wait()));
         }
 
         /// <summary>
@@ -264,16 +187,10 @@ namespace Apache.Ignite.Core.Tests.Compute
         [Test]
         public void TestApplyMultipleReducer()
         {
-            var args = new List<object>(MultiCloCnt);
+            var args = Enumerable.Repeat(1, MultiCloCnt).Cast<object>().ToArray();
 
-            for (int i = 0; i < MultiCloCnt; i++)
-                args.Add(1);
-
-            ICollection<object> ress =
-                Grid1.GetCompute().Apply(Func(false), args, new Reducer(false));
-
-            foreach (object res in ress)
-                CheckResult(res);
+            Grid1.GetCompute().Apply(Func(false), args, new Reducer(false)).ToList().ForEach(CheckResult);
+            Grid1.GetCompute().ApplyAsync(Func(false), args, new Reducer(false)).Result.ToList().ForEach(CheckResult);
         }
 
         /// <summary>
@@ -282,21 +199,10 @@ namespace Apache.Ignite.Core.Tests.Compute
         [Test]
         public void TestAppylMultipleReducerJobException()
         {
-            List<object> args = new List<object>(MultiCloCnt);
+            var args = Enumerable.Repeat(1, MultiCloCnt).Cast<object>().ToArray();
 
-            for (int i = 0; i < MultiCloCnt; i++)
-                args.Add(1);
-
-            try
-            {
-                Grid1.GetCompute().Apply(Func(true), args, new Reducer(false));
-
-                Assert.Fail();
-            }
-            catch (Exception e)
-            {
-                CheckError(e);
-            }
+            CheckError(Assert.Catch(() => Grid1.GetCompute().Apply(Func(true), args, new Reducer(false))));
+            CheckError(Assert.Catch(() => Grid1.GetCompute().ApplyAsync(Func(true), args, new Reducer(false)).Wait()));
         }
 
         /// <summary>
@@ -305,23 +211,11 @@ namespace Apache.Ignite.Core.Tests.Compute
         [Test]
         public void TestAppylMultipleReducerReduceException()
         {
-            var args = new List<object>(MultiCloCnt);
-
-            for (int i = 0; i < MultiCloCnt; i++)
-                args.Add(1);
+            var args = Enumerable.Repeat(1, MultiCloCnt).Cast<object>().ToArray();
 
-            try
-            {
-                Grid1.GetCompute().Apply(Func(false), args, new Reducer(true));
+            var e = Assert.Throws<Exception>(() => Grid1.GetCompute().Apply(Func(false), args, new Reducer(true)));
 
-                Assert.Fail();
-            }
-            catch (Exception e)
-            {
-                Assert.AreEqual(typeof(Exception), e.GetType());
-
-                Assert.AreEqual(ErrMsg, e.Message);
-            }
+            Assert.AreEqual(ErrMsg, e.Message);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
index 20290f1..1e0287f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/ComputeApiTest.cs
@@ -187,7 +187,10 @@ namespace Apache.Ignite.Core.Tests.Compute
 
             Assert.NotNull(prj);
 
-            Assert.IsTrue(prj == prj.Ignite);
+            Assert.AreEqual(prj, prj.Ignite);
+
+            // Check that default Compute projection excludes client nodes.
+            CollectionAssert.AreEquivalent(prj.ForServers().GetNodes(), prj.GetCompute().ClusterGroup.GetNodes());
         }
 
         /// <summary>
@@ -991,9 +994,7 @@ namespace Apache.Ignite.Core.Tests.Compute
 
             compute.WithKeepBinary();
 
-            PlatformComputeNetBinarizable arg = new PlatformComputeNetBinarizable();
-
-            arg.Field = 100;
+            PlatformComputeNetBinarizable arg = new PlatformComputeNetBinarizable {Field = 100};
 
             int res = compute.ExecuteJavaTask<int>(BinaryArgTask, arg);
 
@@ -1036,9 +1037,11 @@ namespace Apache.Ignite.Core.Tests.Compute
         public void TestBroadcastAction()
         {
             var id = Guid.NewGuid();
-            
             _grid1.GetCompute().Broadcast(new ComputeAction(id));
+            Assert.AreEqual(2, ComputeAction.InvokeCount(id));
 
+            id = Guid.NewGuid();
+            _grid1.GetCompute().BroadcastAsync(new ComputeAction(id)).Wait();
             Assert.AreEqual(2, ComputeAction.InvokeCount(id));
         }
 
@@ -1049,9 +1052,11 @@ namespace Apache.Ignite.Core.Tests.Compute
         public void TestRunAction()
         {
             var id = Guid.NewGuid();
-
             _grid1.GetCompute().Run(new ComputeAction(id));
+            Assert.AreEqual(1, ComputeAction.InvokeCount(id));
 
+            id = Guid.NewGuid();
+            _grid1.GetCompute().RunAsync(new ComputeAction(id)).Wait();
             Assert.AreEqual(1, ComputeAction.InvokeCount(id));
         }
 
@@ -1081,12 +1086,12 @@ namespace Apache.Ignite.Core.Tests.Compute
         public void TestRunActions()
         {
             var id = Guid.NewGuid();
-
-            var actions = Enumerable.Range(0, 10).Select(x => new ComputeAction(id));
-            
-            _grid1.GetCompute().Run(actions);
-
+            _grid1.GetCompute().Run(Enumerable.Range(0, 10).Select(x => new ComputeAction(id)));
             Assert.AreEqual(10, ComputeAction.InvokeCount(id));
+
+            var id2 = Guid.NewGuid();
+            _grid1.GetCompute().RunAsync(Enumerable.Range(0, 10).Select(x => new ComputeAction(id2))).Wait();
+            Assert.AreEqual(10, ComputeAction.InvokeCount(id2));
         }
 
         /// <summary>
@@ -1109,7 +1114,9 @@ namespace Apache.Ignite.Core.Tests.Compute
                 var affinityKey = _grid1.GetAffinity(cacheName).GetAffinityKey<int, int>(primaryKey);
 
                 _grid1.GetCompute().AffinityRun(cacheName, affinityKey, new ComputeAction());
+                Assert.AreEqual(node.Id, ComputeAction.LastNodeId);
 
+                _grid1.GetCompute().AffinityRunAsync(cacheName, affinityKey, new ComputeAction()).Wait();
                 Assert.AreEqual(node.Id, ComputeAction.LastNodeId);
             }
         }
@@ -1138,6 +1145,15 @@ namespace Apache.Ignite.Core.Tests.Compute
                 Assert.AreEqual(result, ComputeFunc.InvokeCount);
 
                 Assert.AreEqual(node.Id, ComputeFunc.LastNodeId);
+
+                // Async.
+                ComputeFunc.InvokeCount = 0;
+
+                result = _grid1.GetCompute().AffinityCallAsync(cacheName, affinityKey, new ComputeFunc()).Result;
+
+                Assert.AreEqual(result, ComputeFunc.InvokeCount);
+
+                Assert.AreEqual(node.Id, ComputeFunc.LastNodeId);
             }
         }
 
@@ -1175,10 +1191,18 @@ namespace Apache.Ignite.Core.Tests.Compute
         [Test]
         public void TestNetTaskSimple()
         {
-            int res = _grid1.GetCompute().Execute<NetSimpleJobArgument, NetSimpleJobResult, NetSimpleTaskResult>(
-                    typeof(NetSimpleTask), new NetSimpleJobArgument(1)).Res;
+            Assert.AreEqual(2, _grid1.GetCompute()
+                .Execute<NetSimpleJobArgument, NetSimpleJobResult, NetSimpleTaskResult>(
+                typeof(NetSimpleTask), new NetSimpleJobArgument(1)).Res);
+
+            Assert.AreEqual(2, _grid1.GetCompute()
+                .ExecuteAsync<NetSimpleJobArgument, NetSimpleJobResult, NetSimpleTaskResult>(
+                typeof(NetSimpleTask), new NetSimpleJobArgument(1)).Result.Res);
 
-            Assert.AreEqual(2, res);
+            Assert.AreEqual(4, _grid1.GetCompute().Execute(new NetSimpleTask(), new NetSimpleJobArgument(2)).Res);
+
+            Assert.AreEqual(6, _grid1.GetCompute().ExecuteAsync(new NetSimpleTask(), new NetSimpleJobArgument(3))
+                .Result.Res);
         }
 
         /// <summary>
@@ -1407,6 +1431,7 @@ namespace Apache.Ignite.Core.Tests.Compute
 
         int IComputeFunc<int>.Invoke()
         {
+            Thread.Sleep(10);
             InvokeCount++;
             LastNodeId = _grid.GetCluster().GetLocalNode().Id;
             return InvokeCount;

http://git-wip-us.apache.org/repos/asf/ignite/blob/dc0adf64/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/FailoverTaskSelfTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/FailoverTaskSelfTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/FailoverTaskSelfTest.cs
index 45af888..1987245 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/FailoverTaskSelfTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Compute/FailoverTaskSelfTest.cs
@@ -19,7 +19,6 @@ namespace Apache.Ignite.Core.Tests.Compute
 {
     using System;
     using System.Collections.Generic;
-    using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Compute;
     using Apache.Ignite.Core.Resource;
@@ -111,9 +110,9 @@ namespace Apache.Ignite.Core.Tests.Compute
         }
 
         /** <inheritDoc /> */
-        override protected void GetBinaryTypeConfigurations(ICollection<BinaryTypeConfiguration> portTypeCfgs)
+        protected override ICollection<Type> GetBinaryTypes()
         {
-            portTypeCfgs.Add(new BinaryTypeConfiguration(typeof(TestBinarizableJob)));
+            return new[] {typeof(TestBinarizableJob)};
         }
 
         /// <summary>


[21/50] [abbrv] ignite git commit: IGNITE-4119 .NET: add TransactionDeadlockException

Posted by vo...@apache.org.
IGNITE-4119 .NET: add TransactionDeadlockException


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

Branch: refs/heads/ignite-2693
Commit: 474f22fda4c7cf4d7b2623c451cd7c10f0d8c636
Parents: df670c7
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Nov 7 12:55:20 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Nov 7 12:55:20 2016 +0300

----------------------------------------------------------------------
 .../platform/cache/PlatformCache.java           | 12 ++++
 .../Cache/CacheAbstractTest.cs                  | 44 +++++++++++-
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |  3 +
 .../Apache.Ignite.Core.csproj                   |  3 +-
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |  1 +
 .../TransactionDeadlockException.cs             | 71 ++++++++++++++++++++
 6 files changed, 132 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/474f22fd/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
index be50e5d..6f23682 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
@@ -58,6 +58,8 @@ import org.apache.ignite.internal.util.typedef.C1;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.transactions.TransactionDeadlockException;
+import org.apache.ignite.transactions.TransactionTimeoutException;
 import org.jetbrains.annotations.Nullable;
 
 import javax.cache.Cache;
@@ -1095,6 +1097,16 @@ public class PlatformCache extends PlatformAbstractTarget {
         if (e.getCause() instanceof EntryProcessorException)
             return (Exception)e.getCause();
 
+        TransactionDeadlockException deadlockException = X.cause(e, TransactionDeadlockException.class);
+
+        if (deadlockException != null)
+            return deadlockException;
+
+        TransactionTimeoutException timeoutException = X.cause(e, TransactionTimeoutException.class);
+
+        if (timeoutException != null)
+            return timeoutException;
+
         return super.convertException(e);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/474f22fd/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
index 9fd1f1d..63e236a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
@@ -2458,7 +2458,49 @@ namespace Apache.Ignite.Core.Tests.Cache
                 // Expected
             }
         }
-        
+
+        /// <summary>
+        /// Tests the transaction deadlock detection.
+        /// </summary>
+        [Test]
+        public void TestTxDeadlockDetection()
+        {
+            if (!TxEnabled())
+                return;
+
+            var cache = Cache();
+
+            var keys0 = Enumerable.Range(1, 100).ToArray();
+
+            cache.PutAll(keys0.ToDictionary(x => x, x => x));
+
+            var barrier = new Barrier(2);
+
+            Action<int[]> increment = keys =>
+            {
+                using (var tx = Transactions.TxStart(TransactionConcurrency.Pessimistic,
+                    TransactionIsolation.RepeatableRead, TimeSpan.FromSeconds(0.5), 0))
+                {
+                    foreach (var key in keys)
+                        cache[key]++;
+
+                    barrier.SignalAndWait(500);
+
+                    tx.Commit();
+                }
+            };
+
+            // Increment keys within tx in different order to cause a deadlock.
+            var aex = Assert.Throws<AggregateException>(() =>
+                Task.WaitAll(Task.Factory.StartNew(() => increment(keys0)),
+                             Task.Factory.StartNew(() => increment(keys0.Reverse().ToArray()))));
+
+            Assert.AreEqual(2, aex.InnerExceptions.Count);
+
+            var deadlockEx = aex.InnerExceptions.OfType<TransactionDeadlockException>().First();
+            Assert.IsTrue(deadlockEx.Message.Trim().StartsWith("Deadlock detected:"), deadlockEx.Message);
+        }
+
         /// <summary>
         /// Test thraed-locals leak.
         /// </summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/474f22fd/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs
index e766f5a..052ff6f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs
@@ -65,6 +65,8 @@ namespace Apache.Ignite.Core.Tests
 
             var e = Assert.Throws<ClusterGroupEmptyException>(() => grid.GetCluster().ForRemotes().GetMetrics());
 
+            Assert.IsNotNull(e.InnerException);
+
             Assert.IsTrue(e.InnerException.Message.StartsWith(
                 "class org.apache.ignite.cluster.ClusterGroupEmptyException: Cluster group is empty."));
 
@@ -163,6 +165,7 @@ namespace Apache.Ignite.Core.Tests
 
                 ex = (Exception) msgCauseCtor.Invoke(new object[] {"myMessage", new Exception("innerEx")});
                 Assert.AreEqual("myMessage", ex.Message);
+                Assert.IsNotNull(ex.InnerException);
                 Assert.AreEqual("innerEx", ex.InnerException.Message);
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/474f22fd/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 2973bb3..66253f6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -75,7 +75,7 @@
     <Optimize>true</Optimize>
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>Apache.Ignite.Core.ruleset</CodeAnalysisRuleSet>
-	<DebugType>none</DebugType>
+    <DebugType>none</DebugType>
   </PropertyGroup>
   <ItemGroup>
     <Reference Include="System" />
@@ -477,6 +477,7 @@
     <Compile Include="Transactions\Package-Info.cs" />
     <Compile Include="Transactions\TransactionConcurrency.cs" />
     <Compile Include="Transactions\TransactionConfiguration.cs" />
+    <Compile Include="Transactions\TransactionDeadlockException.cs" />
     <Compile Include="Transactions\TransactionHeuristicException.cs" />
     <Compile Include="Transactions\TransactionIsolation.cs" />
     <Compile Include="Transactions\TransactionOptimisticException.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/474f22fd/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
index a59ca5f..ddbdd86 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
@@ -97,6 +97,7 @@ namespace Apache.Ignite.Core.Impl
             Exs["org.apache.ignite.transactions.TransactionTimeoutException"] = (i, m, e) => new TransactionTimeoutException(m, e);
             Exs["org.apache.ignite.transactions.TransactionRollbackException"] = (i, m, e) => new TransactionRollbackException(m, e);
             Exs["org.apache.ignite.transactions.TransactionHeuristicException"] = (i, m, e) => new TransactionHeuristicException(m, e);
+            Exs["org.apache.ignite.transactions.TransactionDeadlockException"] = (i, m, e) => new TransactionDeadlockException(m, e);
 
             // Security exceptions.
             Exs["org.apache.ignite.IgniteAuthenticationException"] = (i, m, e) => new SecurityException(m, e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/474f22fd/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/TransactionDeadlockException.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/TransactionDeadlockException.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/TransactionDeadlockException.cs
new file mode 100644
index 0000000..b0ba5de
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/TransactionDeadlockException.cs
@@ -0,0 +1,71 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Transactions
+{
+    using System;
+    using System.Runtime.Serialization;
+    using Apache.Ignite.Core.Common;
+
+    /// <summary>
+    /// Indicates a deadlock within Ignite transaction.
+    /// <para />
+    /// This exception can be thrown from any cache method that modifies or reads data within a transaction 
+    /// with timeout (see 
+    /// <see cref="ITransactions.TxStart(TransactionConcurrency, TransactionIsolation, TimeSpan, int)"/> overload).
+    /// </summary>
+    [Serializable]
+    public class TransactionDeadlockException : IgniteException
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionDeadlockException"/> class.
+        /// </summary>
+        public TransactionDeadlockException()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionDeadlockException"/> class.
+        /// </summary>
+        /// <param name="message">The message that describes the error.</param>
+        public TransactionDeadlockException(string message) : base(message)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionDeadlockException"/> class.
+        /// </summary>
+        /// <param name="message">The message.</param>
+        /// <param name="cause">The cause.</param>
+        public TransactionDeadlockException(string message, Exception cause) : base(message, cause)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="TransactionDeadlockException"/> class.
+        /// </summary>
+        /// <param name="info">Serialization information.</param>
+        /// <param name="ctx">Streaming context.</param>
+        protected TransactionDeadlockException(SerializationInfo info, StreamingContext ctx) : base(info, ctx)
+        {
+            // No-op.
+        }
+    }
+}


[07/50] [abbrv] ignite git commit: IGNITE-3711 .NET: Fix PlatformRawMemory coverage issue

Posted by vo...@apache.org.
IGNITE-3711 .NET: Fix PlatformRawMemory coverage issue


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

Branch: refs/heads/ignite-2693
Commit: cd0275d95dc6524464a154c74b1e62559b3d8af4
Parents: dc0adf6
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Aug 19 17:34:00 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Nov 1 19:26:43 2016 +0300

----------------------------------------------------------------------
 .../Impl/Memory/PlatformRawMemory.cs                  |  1 +
 .../Impl/Unmanaged/UnmanagedCallbacks.cs              | 14 +++++---------
 2 files changed, 6 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cd0275d9/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs
index f252ef3..c3cbee2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs
@@ -24,6 +24,7 @@ namespace Apache.Ignite.Core.Impl.Memory
     /// Non-resizeable raw memory chunk without metadata header.
     /// </summary>
     [CLSCompliant(false)]
+    [ExcludeFromCodeCoverage]
     public class PlatformRawMemory : IPlatformMemory
     {
         /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cd0275d9/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
index fcc1652..95a4633 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
@@ -1097,6 +1097,11 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
         private void Error(void* target, int errType, sbyte* errClsChars, int errClsCharsLen, sbyte* errMsgChars,
             int errMsgCharsLen, sbyte* stackTraceChars, int stackTraceCharsLen, void* errData, int errDataLen)
         {
+            // errData mechanism is only needed for CachePartialUpdateException and is no longer used,
+            // since CacheImpl handles all errors itself.
+            Debug.Assert(errDataLen == 0);
+            Debug.Assert(errData == null);
+
             string errCls = IgniteUtils.Utf8UnmanagedToString(errClsChars, errClsCharsLen);
             string errMsg = IgniteUtils.Utf8UnmanagedToString(errMsgChars, errMsgCharsLen);
             string stackTrace = IgniteUtils.Utf8UnmanagedToString(stackTraceChars, stackTraceCharsLen);
@@ -1104,15 +1109,6 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
             switch (errType)
             {
                 case ErrGeneric:
-                    if (_ignite != null && errDataLen > 0)
-                    {
-                        // Stream disposal intentionally omitted: IGNITE-1598
-                        var stream = new PlatformRawMemory(errData, errDataLen).GetStream();
-
-                        throw ExceptionUtils.GetException(_ignite, errCls, errMsg, stackTrace,
-                            _ignite.Marshaller.StartUnmarshal(stream));
-                    }
-
                     throw ExceptionUtils.GetException(_ignite, errCls, errMsg, stackTrace);
 
                 case ErrJvmInit:


[31/50] [abbrv] ignite git commit: ignite-4176 IgniteQueue not functioning as expected after remove()

Posted by vo...@apache.org.
ignite-4176 IgniteQueue not functioning as expected after remove()


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

Branch: refs/heads/ignite-2693
Commit: 51e96d614de1588843ef4374cbbb91c426297c3e
Parents: 703957d
Author: Amir Akhmedov <am...@gmail.com>
Authored: Wed Nov 9 13:07:13 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Nov 9 13:07:13 2016 +0300

----------------------------------------------------------------------
 .../datastructures/GridCacheQueueAdapter.java   |  2 +-
 .../GridCacheQueueApiSelfAbstractTest.java      | 66 ++++++++++++++++++++
 2 files changed, 67 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/51e96d61/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java
index caf3ba3..6e087e6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java
@@ -834,7 +834,7 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
                 return hdr.head();
             }
 
-            long next = hdr.head() + 1;
+            long next = hdr.head();
 
             rmvdIdxs = new HashSet<>(rmvdIdxs);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/51e96d61/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java
index f9499a1..93d0989 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java
@@ -534,6 +534,72 @@ public abstract class GridCacheQueueApiSelfAbstractTest extends IgniteCollection
     }
 
     /**
+     * JUnit.
+     *
+     * @throws Exception If failed.
+     */
+    public void testPutRemovePeekPollUnbounded() throws Exception {
+        // Random queue name.
+        String queueName = UUID.randomUUID().toString();
+
+        IgniteQueue<String> queue = grid(0).queue(queueName, 0, config(false));
+
+        for (int i = 0; i < QUEUE_CAPACITY; i++)
+            queue.put("Item-" + i);
+
+        assertEquals(QUEUE_CAPACITY, queue.size());
+
+        queue.remove("Item-1");
+
+        assertEquals(QUEUE_CAPACITY - 1, queue.size());
+
+        assertEquals("Item-0", queue.peek());
+        assertEquals("Item-0", queue.poll());
+        assertEquals("Item-2", queue.poll());
+
+        assertEquals(0, queue.size());
+
+        queue.clear();
+
+        assertTrue(queue.isEmpty());
+    }
+
+    /**
+     * JUnit.
+     *
+     * @throws Exception If failed.
+     */
+    public void testRemovePeek() throws Exception {
+        // Random queue name.
+        String queueName = UUID.randomUUID().toString();
+
+        IgniteQueue<String> queue = grid(0).queue(queueName, 0, config(false));
+
+        for (int i = 0; i < 5; i++)
+            queue.put("Item-" + i);
+
+        queue.remove("Item-1");
+
+        assertEquals("Item-0", queue.peek());
+
+        queue.remove("Item-2");
+
+        assertEquals("Item-0", queue.peek());
+
+        queue.remove("Item-0");
+
+        assertEquals("Item-3", queue.peek());
+
+        queue.remove("Item-4");
+
+        assertEquals("Item-3", queue.peek());
+
+        queue.remove("Item-3");
+
+        assertNull(queue.peek());
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testReuseCache() throws Exception {


[32/50] [abbrv] ignite git commit: ignite-3812 added thread pool size validation

Posted by vo...@apache.org.
ignite-3812 added thread pool size validation


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

Branch: refs/heads/ignite-2693
Commit: 2bc234ed8bec13998fdb2f6eeb4e8703263be21c
Parents: 51e96d6
Author: Alexandr Kuramshin <ak...@gridgain.com>
Authored: Wed Nov 9 16:15:01 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Nov 9 16:15:01 2016 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/IgnitionEx.java  |  32 ++++-
 .../testsuites/IgniteUtilSelfTestSuite.java     |   2 +
 .../ignite/thread/IgniteThreadPoolSizeTest.java | 131 +++++++++++++++++++
 3 files changed, 164 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2bc234ed/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 5b2c3fc..b3a9eec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1640,6 +1640,8 @@ public class IgnitionEx {
                 ensureMultiInstanceSupport(myCfg.getSwapSpaceSpi());
             }
 
+            validateThreadPoolSize(cfg.getPublicThreadPoolSize(), "public");
+
             execSvc = new IgniteThreadPoolExecutor(
                 "pub",
                 cfg.getGridName(),
@@ -1652,6 +1654,8 @@ public class IgnitionEx {
 
             // Note that since we use 'LinkedBlockingQueue', number of
             // maximum threads has no effect.
+            validateThreadPoolSize(cfg.getSystemThreadPoolSize(), "system");
+
             sysExecSvc = new IgniteThreadPoolExecutor(
                 "sys",
                 cfg.getGridName(),
@@ -1666,6 +1670,8 @@ public class IgnitionEx {
             // maximum threads has no effect.
             // Note, that we do not pre-start threads here as management pool may
             // not be needed.
+            validateThreadPoolSize(cfg.getManagementThreadPoolSize(), "management");
+
             mgmtExecSvc = new IgniteThreadPoolExecutor(
                 "mgmt",
                 cfg.getGridName(),
@@ -1680,6 +1686,7 @@ public class IgnitionEx {
             // maximum threads has no effect.
             // Note, that we do not pre-start threads here as class loading pool may
             // not be needed.
+            validateThreadPoolSize(cfg.getPeerClassLoadingThreadPoolSize(), "peer class loading");
             p2pExecSvc = new IgniteThreadPoolExecutor(
                 "p2p",
                 cfg.getGridName(),
@@ -1691,6 +1698,8 @@ public class IgnitionEx {
             p2pExecSvc.allowCoreThreadTimeOut(true);
 
             // Note that we do not pre-start threads here as igfs pool may not be needed.
+            validateThreadPoolSize(cfg.getIgfsThreadPoolSize(), "IGFS");
+
             igfsExecSvc = new IgniteThreadPoolExecutor(
                 cfg.getIgfsThreadPoolSize(),
                 cfg.getIgfsThreadPoolSize(),
@@ -1702,12 +1711,16 @@ public class IgnitionEx {
             igfsExecSvc.allowCoreThreadTimeOut(true);
 
             // Note that we do not pre-start threads here as this pool may not be needed.
+            validateThreadPoolSize(cfg.getAsyncCallbackPoolSize(), "async callback");
+
             callbackExecSvc = new IgniteStripedThreadPoolExecutor(
                 cfg.getAsyncCallbackPoolSize(),
                 cfg.getGridName(),
                 "callback");
 
             if (myCfg.getConnectorConfiguration() != null) {
+                validateThreadPoolSize(myCfg.getConnectorConfiguration().getThreadPoolSize(), "connector");
+
                 restExecSvc = new IgniteThreadPoolExecutor(
                     "rest",
                     myCfg.getGridName(),
@@ -1720,6 +1733,8 @@ public class IgnitionEx {
                 restExecSvc.allowCoreThreadTimeOut(true);
             }
 
+            validateThreadPoolSize(myCfg.getUtilityCacheThreadPoolSize(), "utility cache");
+
             utilityCacheExecSvc = new IgniteThreadPoolExecutor(
                 "utility",
                 cfg.getGridName(),
@@ -1730,6 +1745,8 @@ public class IgnitionEx {
 
             utilityCacheExecSvc.allowCoreThreadTimeOut(true);
 
+            validateThreadPoolSize(myCfg.getMarshallerCacheThreadPoolSize(), "marshaller cache");
+
             marshCacheExecSvc = new IgniteThreadPoolExecutor(
                 "marshaller-cache",
                 cfg.getGridName(),
@@ -1838,6 +1855,19 @@ public class IgnitionEx {
         }
 
         /**
+         * @param poolSize an actual value in the configuration.
+         * @param poolName a name of the pool like 'management'.
+         * @throws IgniteCheckedException If the poolSize is wrong.
+         */
+        private static void validateThreadPoolSize(int poolSize, String poolName)
+            throws IgniteCheckedException {
+            if (poolSize <= 0) {
+                throw new IgniteCheckedException("Invalid " + poolName + " thread pool size" +
+                    " (must be greater than 0), actual value: " + poolSize);
+            }
+        }
+
+        /**
          * @param cfg Ignite configuration copy to.
          * @return New ignite configuration.
          * @throws IgniteCheckedException If failed.
@@ -2019,7 +2049,7 @@ public class IgnitionEx {
             if (userCaches != null && userCaches.length > 0) {
                 if (!U.discoOrdered(cfg.getDiscoverySpi()) && !U.relaxDiscoveryOrdered())
                     throw new IgniteCheckedException("Discovery SPI implementation does not support node ordering and " +
-                        "cannot be used with cache (use SPI with @GridDiscoverySpiOrderSupport annotation, " +
+                        "cannot be used with cache (use SPI with @DiscoverySpiOrderSupport annotation, " +
                         "like TcpDiscoverySpi)");
 
                 for (CacheConfiguration ccfg : userCaches) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/2bc234ed/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
index bd7bb96..078b865 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
@@ -36,6 +36,7 @@ import org.apache.ignite.lang.GridByteArrayListSelfTest;
 import org.apache.ignite.spi.discovery.ClusterMetricsSnapshotSerializeSelfTest;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.thread.GridThreadPoolExecutorServiceSelfTest;
+import org.apache.ignite.thread.IgniteThreadPoolSizeTest;
 import org.apache.ignite.util.GridLongListSelfTest;
 import org.apache.ignite.util.GridMessageCollectionTest;
 import org.apache.ignite.util.GridQueueSelfTest;
@@ -64,6 +65,7 @@ public class IgniteUtilSelfTestSuite extends TestSuite {
         TestSuite suite = new TestSuite("Ignite Util Test Suite");
 
         suite.addTestSuite(GridThreadPoolExecutorServiceSelfTest.class);
+        suite.addTestSuite(IgniteThreadPoolSizeTest.class);
         GridTestUtils.addTestIfNeeded(suite, IgniteUtilsSelfTest.class, ignoredTests);
         suite.addTestSuite(GridSpinReadWriteLockSelfTest.class);
         suite.addTestSuite(GridQueueSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/2bc234ed/modules/core/src/test/java/org/apache/ignite/thread/IgniteThreadPoolSizeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/thread/IgniteThreadPoolSizeTest.java b/modules/core/src/test/java/org/apache/ignite/thread/IgniteThreadPoolSizeTest.java
new file mode 100644
index 0000000..d1fd4e7
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/thread/IgniteThreadPoolSizeTest.java
@@ -0,0 +1,131 @@
+/*
+ * 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.thread;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class IgniteThreadPoolSizeTest extends GridCommonAbstractTest {
+    /** Wrong thread pool size value for testing */
+    private static final int WRONG_VALUE = 0;
+
+    /**
+     * @return Ignite configuration.
+     */
+    private IgniteConfiguration configuration() {
+        return new IgniteConfiguration();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAsyncCallbackPoolSize() throws Exception {
+        testWrongPoolSize(configuration().setAsyncCallbackPoolSize(WRONG_VALUE));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIgfsThreadPoolSize() throws Exception {
+        testWrongPoolSize(configuration().setIgfsThreadPoolSize(WRONG_VALUE));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testManagementThreadPoolSize() throws Exception {
+        testWrongPoolSize(configuration().setManagementThreadPoolSize(WRONG_VALUE));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPeerClassLoadingThreadPoolSize() throws Exception {
+        testWrongPoolSize(configuration().setPeerClassLoadingThreadPoolSize(WRONG_VALUE));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPublicThreadPoolSize() throws Exception {
+        testWrongPoolSize(configuration().setPublicThreadPoolSize(WRONG_VALUE));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRebalanceThreadPoolSize() throws Exception {
+        testWrongPoolSize(configuration().setRebalanceThreadPoolSize(WRONG_VALUE));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSystemThreadPoolSize() throws Exception {
+        testWrongPoolSize(configuration().setSystemThreadPoolSize(WRONG_VALUE));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUtilityCachePoolSize() throws Exception {
+        testWrongPoolSize(configuration().setUtilityCachePoolSize(WRONG_VALUE));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("deprecated")
+    public void testMarshallerCachePoolSize() throws Exception {
+        testWrongPoolSize(configuration().setMarshallerCachePoolSize(WRONG_VALUE));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectorThreadPoolSize() throws Exception {
+        final IgniteConfiguration cfg = configuration();
+
+        cfg.getConnectorConfiguration().setThreadPoolSize(WRONG_VALUE);
+
+        testWrongPoolSize(cfg);
+    }
+
+    /**
+     * Performs testing for wrong tread pool size.
+     *
+     * @param cfg an IgniteConfiguration with the only one thread pool size assigned with the WRONG_VALUE.
+     * @throws Exception If failed.
+     */
+    private void testWrongPoolSize(IgniteConfiguration cfg) throws Exception {
+        try {
+            Ignition.start(cfg);
+
+            fail();
+        }
+        catch (IgniteException ex) {
+            assertNotNull(ex.getMessage());
+            assertTrue(ex.getMessage().contains("thread pool size"));
+        }
+    }
+}


[16/50] [abbrv] ignite git commit: Add test case for testing permits releasing by IgniteSempahore with failoverSafe enabled

Posted by vo...@apache.org.
Add test case for testing permits releasing by IgniteSempahore with failoverSafe enabled


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

Branch: refs/heads/ignite-2693
Commit: 138a3aad20f7dcb16a6c7e6d047d0503450ea06e
Parents: e1defc0
Author: agura <ag...@apache.org>
Authored: Thu Nov 3 15:40:01 2016 +0300
Committer: agura <ag...@apache.org>
Committed: Thu Nov 3 15:40:01 2016 +0300

----------------------------------------------------------------------
 ...SemaphoreFailoverSafeReleasePermitsTest.java | 129 +++++++++++++++++++
 .../IgniteCacheDataStructuresSelfTestSuite.java |   2 +
 2 files changed, 131 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/138a3aad/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/SemaphoreFailoverSafeReleasePermitsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/SemaphoreFailoverSafeReleasePermitsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/SemaphoreFailoverSafeReleasePermitsTest.java
new file mode 100644
index 0000000..241253d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/SemaphoreFailoverSafeReleasePermitsTest.java
@@ -0,0 +1,129 @@
+/*
+ * 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.datastructures;
+
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteSemaphore;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.AtomicConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+
+/**
+ *
+ */
+public class SemaphoreFailoverSafeReleasePermitsTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** Grid count. */
+    private static final int GRID_CNT = 3;
+
+    /** Atomics cache mode. */
+    private CacheMode atomicsCacheMode;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi spi = new TcpDiscoverySpi();
+
+        spi.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(spi);
+
+        AtomicConfiguration atomicCfg = atomicConfiguration();
+
+        assertNotNull(atomicCfg);
+
+        cfg.setAtomicConfiguration(atomicCfg);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReleasePermitsPartitioned() throws Exception {
+        atomicsCacheMode = PARTITIONED;
+
+        doTest();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReleasePermitsReplicated() throws Exception {
+        atomicsCacheMode = REPLICATED;
+
+        doTest();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void doTest() throws Exception {
+        try {
+            startGrids(GRID_CNT);
+
+            Ignite ignite = grid(0);
+
+            IgniteSemaphore sem = ignite.semaphore("sem", 1, true, true);
+
+            assertEquals(1, sem.availablePermits());
+
+            sem.acquire(1);
+
+            assertEquals(0, sem.availablePermits());
+
+            ignite.close();
+
+            awaitPartitionMapExchange();
+
+            ignite = grid(1);
+
+            sem = ignite.semaphore("sem", 1, true, true);
+
+            assertTrue(sem.tryAcquire(1, 5000, TimeUnit.MILLISECONDS));
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @return Atomic configuration.
+     */
+    protected AtomicConfiguration atomicConfiguration() {
+        AtomicConfiguration atomicCfg = new AtomicConfiguration();
+
+        atomicCfg.setCacheMode(atomicsCacheMode);
+
+        if (atomicsCacheMode == PARTITIONED)
+            atomicCfg.setBackups(1);
+
+        return atomicCfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/138a3aad/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
index d62369c..45a49bf 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
@@ -24,6 +24,7 @@ import org.apache.ignite.internal.processors.cache.datastructures.IgniteClientDa
 import org.apache.ignite.internal.processors.cache.datastructures.IgniteClientDiscoveryDataStructuresTest;
 import org.apache.ignite.internal.processors.cache.datastructures.IgniteDataStructureUniqueNameTest;
 import org.apache.ignite.internal.processors.cache.datastructures.IgniteDataStructureWithJobTest;
+import org.apache.ignite.internal.processors.cache.datastructures.SemaphoreFailoverSafeReleasePermitsTest;
 import org.apache.ignite.internal.processors.cache.datastructures.local.GridCacheLocalAtomicOffheapSetSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.local.GridCacheLocalAtomicQueueApiSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.local.GridCacheLocalAtomicSetSelfTest;
@@ -144,6 +145,7 @@ public class IgniteCacheDataStructuresSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(IgnitePartitionedCountDownLatchSelfTest.class));
         suite.addTest(new TestSuite(IgniteDataStructureWithJobTest.class));
         suite.addTest(new TestSuite(IgnitePartitionedSemaphoreSelfTest.class));
+        suite.addTest(new TestSuite(SemaphoreFailoverSafeReleasePermitsTest.class));
         // TODO IGNITE-3141, enabled when fixed.
         // suite.addTest(new TestSuite(IgnitePartitionedLockSelfTest.class));
 


[23/50] [abbrv] ignite git commit: IGNITE-4132 .NET: Improve BinaryConfiguration documentation

Posted by vo...@apache.org.
IGNITE-4132 .NET: Improve BinaryConfiguration documentation


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

Branch: refs/heads/ignite-2693
Commit: 319014de075c80fb15e58172cc24e35ce16b56cf
Parents: 474f22f
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Nov 7 17:53:40 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Nov 7 17:53:40 2016 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs     | 5 ++++-
 .../dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd    | 2 +-
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/319014de/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs
index 3a9d86c..51df907 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs
@@ -88,7 +88,10 @@ namespace Apache.Ignite.Core.Binary
         public ICollection<BinaryTypeConfiguration> TypeConfigurations { get; set; }
 
         /// <summary>
-        /// Binarizable types. Shorthand for creating <see cref="BinaryTypeConfiguration"/>.
+        /// Gets or sets a collection of assembly-qualified type names 
+        /// (the result of <see cref="Type.AssemblyQualifiedName"/>) for binarizable types.
+        /// <para />
+        /// Shorthand for creating <see cref="BinaryTypeConfiguration"/>.
         /// </summary>
         [SuppressMessage("Microsoft.Usage", "CA2227:CollectionPropertiesShouldBeReadOnly")]
         public ICollection<string> Types { get; set; }

http://git-wip-us.apache.org/repos/asf/ignite/blob/319014de/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
index 8764b60..e9fb991 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -183,7 +183,7 @@
                             <xs:element name="types" minOccurs="0">
                                 <xs:annotation>
                                     <xs:documentation>
-                                        Binarizable types. Shorthand for creating "BinaryTypeConfiguration.
+                                        Assembly-qualified type names (Type.AssemblyQualifiedName) for binarizable types. Shorthand for creating BinaryTypeConfiguration.
                                     </xs:documentation>
                                 </xs:annotation>
                                 <xs:complexType>


[29/50] [abbrv] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by vo...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/ignite-2693
Commit: 44d07905df3c94934c62f14a874c03f87cf9226a
Parents: 173e2b8 e031498
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Nov 9 11:14:18 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Nov 9 11:14:18 2016 +0300

----------------------------------------------------------------------
 .../store/cassandra/CassandraCacheStore.java    |   5 +-
 .../ignite/tests/IgnitePersistentStoreTest.java |  11 +
 .../src/main/java/org/apache/ignite/Ignite.java |  77 ++++-
 .../apache/ignite/internal/IgniteKernal.java    |  81 ++++++
 .../processors/cache/GridCacheProcessor.java    | 291 ++++++++++++++-----
 .../cache/IgniteDynamicCacheStartSelfTest.java  | 217 ++++++++++++--
 .../cache/WithKeepBinaryCacheFullApiTest.java   |  10 +-
 ...SemaphoreFailoverSafeReleasePermitsTest.java | 129 ++++++++
 ...artitionedBackupNodeFailureRecoveryTest.java | 193 ++++++++++++
 .../processors/igfs/IgfsIgniteMock.java         |  19 ++
 .../ignite/testframework/junits/IgniteMock.java |  15 +
 .../junits/multijvm/IgniteProcessProxy.java     |  15 +
 .../IgniteCacheDataStructuresSelfTestSuite.java |   2 +
 .../testsuites/IgniteCacheTestSuite2.java       |   2 +
 .../org/apache/ignite/IgniteSpringBean.java     |  21 ++
 15 files changed, 958 insertions(+), 130 deletions(-)
----------------------------------------------------------------------


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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/44d07905/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
----------------------------------------------------------------------


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

Posted by vo...@apache.org.
Merge branch 'master' into ignite-2693

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java


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

Branch: refs/heads/ignite-2693
Commit: c5390868f57d96e5217f004ae6ac5016f185175d
Parents: 7fd543c dd20d94
Author: devozerov <vo...@gridgain.com>
Authored: Thu Nov 17 13:50:11 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Nov 17 13:50:11 2016 +0300

----------------------------------------------------------------------
 .gitignore                                      |     1 +
 RELEASE_NOTES.txt                               |    19 +
 assembly/LICENSE_FABRIC                         |     3 +-
 assembly/LICENSE_HADOOP                         |     1 -
 assembly/libs/README.txt                        |    39 +-
 bin/include/hadoop-classpath.bat                |    73 -
 bin/include/hadoop-classpath.sh                 |    65 -
 bin/include/setenv.bat                          |    10 -
 bin/include/setenv.sh                           |    12 -
 examples/pom.xml                                |     2 +-
 examples/schema-import/pom.xml                  |     2 +-
 .../hibernate/HibernateL2CacheExample.java      |     2 +-
 .../examples/datagrid/CacheQueryExample.java    |    78 +-
 .../examples/IndexingBridgeMethodTest.java      |    93 +
 .../IgniteExamplesJ8SelfTestSuite.java          |     2 +
 modules/aop/pom.xml                             |     2 +-
 modules/apache-license-gen/pom.xml              |     2 +-
 modules/aws/pom.xml                             |     2 +-
 .../s3/S3CheckpointManagerSelfTest.java         |     3 +
 .../checkpoint/s3/S3CheckpointSpiSelfTest.java  |     4 +
 .../s3/S3CheckpointSpiStartStopSelfTest.java    |     7 +
 .../s3/S3SessionCheckpointSelfTest.java         |     2 +
 .../s3/TcpDiscoveryS3IpFinderSelfTest.java      |     7 +
 .../ignite/testsuites/IgniteS3TestSuite.java    |    15 +-
 modules/benchmarks/pom.xml                      |     6 +-
 .../jmh/notify/JmhParkVsNotifyBenchmark.java    |   105 +
 .../jmh/notify/JmhWaitStategyBenchmark.java     |   259 +
 modules/camel/pom.xml                           |     2 +-
 modules/cassandra/README.txt                    |     8 +-
 modules/cassandra/licenses/apache-2.0.txt       |   202 -
 modules/cassandra/pom.xml                       |   324 +-
 modules/cassandra/serializers/README.txt        |    33 +
 .../serializers/licenses/apache-2.0.txt         |   202 +
 modules/cassandra/serializers/pom.xml           |   129 +
 .../cassandra/serializer/KryoSerializer.java    |    95 +
 .../apache/ignite/tests/KryoSerializerTest.java |    70 +
 .../java/org/apache/ignite/tests/MyPojo.java    |   102 +
 .../store/cassandra/CassandraCacheStore.java    |   409 -
 .../cassandra/CassandraCacheStoreFactory.java   |   200 -
 .../cassandra/bean/CassandraLifeCycleBean.java  |   149 -
 .../store/cassandra/common/CassandraHelper.java |   133 -
 .../cassandra/common/PropertyMappingHelper.java |   220 -
 .../store/cassandra/common/RandomSleeper.java   |   104 -
 .../store/cassandra/common/SystemHelper.java    |    46 -
 .../store/cassandra/datasource/Credentials.java |    37 -
 .../store/cassandra/datasource/DataSource.java  |   550 -
 .../cassandra/datasource/PlainCredentials.java  |    50 -
 .../persistence/KeyPersistenceSettings.java     |   274 -
 .../KeyValuePersistenceSettings.java            |   478 -
 .../persistence/PersistenceController.java      |   421 -
 .../persistence/PersistenceSettings.java        |   335 -
 .../persistence/PersistenceStrategy.java        |    62 -
 .../store/cassandra/persistence/PojoField.java  |   219 -
 .../cassandra/persistence/PojoKeyField.java     |    91 -
 .../cassandra/persistence/PojoValueField.java   |   152 -
 .../persistence/ValuePersistenceSettings.java   |   107 -
 .../cassandra/serializer/JavaSerializer.java    |    81 -
 .../cassandra/serializer/KryoSerializer.java    |    93 -
 .../store/cassandra/serializer/Serializer.java  |    43 -
 .../session/BatchExecutionAssistant.java        |    95 -
 .../cassandra/session/BatchLoaderAssistant.java |    47 -
 .../cassandra/session/CassandraSession.java     |    60 -
 .../cassandra/session/CassandraSessionImpl.java |   832 --
 .../cassandra/session/ExecutionAssistant.java   |    77 -
 .../session/GenericBatchExecutionAssistant.java |    71 -
 .../session/LoadCacheCustomQueryWorker.java     |   105 -
 .../cassandra/session/pool/SessionPool.java     |   173 -
 .../cassandra/session/pool/SessionWrapper.java  |    71 -
 .../store/cassandra/utils/DDLGenerator.java     |    64 -
 .../aws/cassandra/cassandra-bootstrap.sh        |   373 -
 .../bootstrap/aws/cassandra/cassandra-env.sh    |   283 -
 .../bootstrap/aws/cassandra/cassandra-start.sh  |   550 -
 .../aws/cassandra/cassandra-template.yaml       |   889 --
 .../bootstrap/aws/ignite/ignite-bootstrap.sh    |   384 -
 .../ignite/ignite-cassandra-server-template.xml |   177 -
 .../src/test/bootstrap/aws/ignite/ignite-env.sh |    25 -
 .../test/bootstrap/aws/ignite/ignite-start.sh   |   637 -
 .../src/test/bootstrap/aws/logs-collector.sh    |   102 -
 .../tests/ignite-cassandra-client-template.xml  |   173 -
 .../test/bootstrap/aws/tests/tests-bootstrap.sh |   379 -
 .../test/bootstrap/aws/tests/tests-report.sh    |   590 -
 .../src/test/bootstrap/aws/tests/tests-run.sh   |   715 -
 .../CassandraDirectPersistenceLoadTest.java     |   107 -
 .../tests/CassandraDirectPersistenceTest.java   |   371 -
 .../apache/ignite/tests/DDLGeneratorTest.java   |    43 -
 .../tests/IgnitePersistentStoreLoadTest.java    |   111 -
 .../ignite/tests/IgnitePersistentStoreTest.java |   369 -
 .../org/apache/ignite/tests/load/Generator.java |    27 -
 .../apache/ignite/tests/load/IntGenerator.java  |    33 -
 .../ignite/tests/load/LoadTestDriver.java       |   238 -
 .../apache/ignite/tests/load/LongGenerator.java |    28 -
 .../ignite/tests/load/PersonGenerator.java      |    43 -
 .../ignite/tests/load/PersonIdGenerator.java    |    31 -
 .../ignite/tests/load/StringGenerator.java      |    28 -
 .../org/apache/ignite/tests/load/Worker.java    |   429 -
 .../tests/load/cassandra/BulkReadWorker.java    |    63 -
 .../tests/load/cassandra/BulkWriteWorker.java   |    52 -
 .../ignite/tests/load/cassandra/ReadWorker.java |    51 -
 .../tests/load/cassandra/WriteWorker.java       |    51 -
 .../tests/load/ignite/BulkReadWorker.java       |    52 -
 .../tests/load/ignite/BulkWriteWorker.java      |    52 -
 .../ignite/tests/load/ignite/ReadWorker.java    |    51 -
 .../ignite/tests/load/ignite/WriteWorker.java   |    51 -
 .../org/apache/ignite/tests/pojos/Person.java   |   246 -
 .../org/apache/ignite/tests/pojos/PersonId.java |   110 -
 .../ignite/tests/utils/CacheStoreHelper.java    |    64 -
 .../tests/utils/CassandraAdminCredentials.java  |    36 -
 .../ignite/tests/utils/CassandraHelper.java     |   358 -
 .../utils/CassandraRegularCredentials.java      |    36 -
 .../ignite/tests/utils/TestCacheSession.java    |    91 -
 .../apache/ignite/tests/utils/TestsHelper.java  |   375 -
 .../src/test/resources/log4j.properties         |   119 -
 .../tests/cassandra/connection-settings.xml     |    48 -
 .../tests/cassandra/connection.properties       |    17 -
 .../tests/cassandra/credentials.properties      |    22 -
 .../tests/cassandra/embedded-cassandra.yaml     |   119 -
 .../ignite/tests/cassandra/keyspaces.properties |    17 -
 .../tests/persistence/blob/ignite-config.xml    |   100 -
 .../persistence/blob/persistence-settings-1.xml |    21 -
 .../persistence/blob/persistence-settings-2.xml |    21 -
 .../persistence/blob/persistence-settings-3.xml |    29 -
 .../tests/persistence/pojo/ignite-config.xml    |   119 -
 .../persistence/pojo/persistence-settings-1.xml |    21 -
 .../persistence/pojo/persistence-settings-2.xml |    21 -
 .../persistence/pojo/persistence-settings-3.xml |   173 -
 .../persistence/primitive/ignite-config.xml     |   100 -
 .../primitive/ignite-remote-client-config.xml   |    95 -
 .../primitive/ignite-remote-server-config.xml   |   106 -
 .../primitive/persistence-settings-1.xml        |    21 -
 .../primitive/persistence-settings-2.xml        |    21 -
 .../src/test/resources/tests.properties         |    50 -
 .../src/test/scripts/cassandra-load-tests.bat   |    41 -
 .../src/test/scripts/cassandra-load-tests.sh    |    39 -
 .../src/test/scripts/ignite-load-tests.bat      |    41 -
 .../src/test/scripts/ignite-load-tests.sh       |    39 -
 modules/cassandra/src/test/scripts/jvm-opt.sh   |    21 -
 modules/cassandra/src/test/scripts/jvm-opts.bat |    24 -
 modules/cassandra/store/README.txt              |    32 +
 modules/cassandra/store/licenses/apache-2.0.txt |   202 +
 modules/cassandra/store/pom.xml                 |   305 +
 .../store/cassandra/CassandraCacheStore.java    |   519 +
 .../cassandra/CassandraCacheStoreFactory.java   |   200 +
 .../store/cassandra/common/CassandraHelper.java |   176 +
 .../cassandra/common/PropertyMappingHelper.java |   219 +
 .../store/cassandra/common/RandomSleeper.java   |   104 +
 .../store/cassandra/common/SystemHelper.java    |    46 +
 .../store/cassandra/common/package-info.java    |    21 +
 .../store/cassandra/datasource/Credentials.java |    39 +
 .../store/cassandra/datasource/DataSource.java  |   647 +
 .../cassandra/datasource/PlainCredentials.java  |    53 +
 .../cassandra/datasource/package-info.java      |    21 +
 .../cache/store/cassandra/package-info.java     |    21 +
 .../persistence/KeyPersistenceSettings.java     |   305 +
 .../KeyValuePersistenceSettings.java            |   526 +
 .../persistence/PersistenceController.java      |   462 +
 .../persistence/PersistenceSettings.java        |   446 +
 .../persistence/PersistenceStrategy.java        |    62 +
 .../store/cassandra/persistence/PojoField.java  |   245 +
 .../cassandra/persistence/PojoKeyField.java     |    87 +
 .../cassandra/persistence/PojoValueField.java   |   143 +
 .../persistence/ValuePersistenceSettings.java   |   120 +
 .../cassandra/persistence/package-info.java     |    21 +
 .../cassandra/serializer/JavaSerializer.java    |    80 +
 .../store/cassandra/serializer/Serializer.java  |    43 +
 .../cassandra/serializer/package-info.java      |    21 +
 .../session/BatchExecutionAssistant.java        |   102 +
 .../cassandra/session/BatchLoaderAssistant.java |    47 +
 .../cassandra/session/CassandraSession.java     |    70 +
 .../cassandra/session/CassandraSessionImpl.java |   943 ++
 .../cassandra/session/ExecutionAssistant.java   |    84 +
 .../session/GenericBatchExecutionAssistant.java |    71 +
 .../session/LoadCacheCustomQueryWorker.java     |   105 +
 .../store/cassandra/session/package-info.java   |    21 +
 .../cassandra/session/pool/SessionPool.java     |   173 +
 .../cassandra/session/pool/SessionWrapper.java  |    71 +
 .../cassandra/session/pool/package-info.java    |    21 +
 .../session/transaction/BaseMutation.java       |    68 +
 .../session/transaction/DeleteMutation.java     |    57 +
 .../cassandra/session/transaction/Mutation.java |    63 +
 .../session/transaction/WriteMutation.java      |    60 +
 .../session/transaction/package-info.java       |    21 +
 .../store/cassandra/utils/DDLGenerator.java     |    76 +
 .../store/cassandra/utils/package-info.java     |    21 +
 .../store/src/test/bootstrap/aws/README.txt     |    13 +
 .../aws/cassandra/cassandra-bootstrap.sh        |   336 +
 .../bootstrap/aws/cassandra/cassandra-env.sh    |   287 +
 .../bootstrap/aws/cassandra/cassandra-start.sh  |   217 +
 .../aws/cassandra/cassandra-template.yaml       |   888 ++
 .../store/src/test/bootstrap/aws/common.sh      |  1481 ++
 .../store/src/test/bootstrap/aws/env.sh         |   113 +
 .../test/bootstrap/aws/ganglia/agent-start.sh   |    75 +
 .../bootstrap/aws/ganglia/ganglia-bootstrap.sh  |   417 +
 .../bootstrap/aws/ignite/ignite-bootstrap.sh    |   336 +
 .../ignite/ignite-cassandra-server-template.xml |   190 +
 .../src/test/bootstrap/aws/ignite/ignite-env.sh |    29 +
 .../test/bootstrap/aws/ignite/ignite-start.sh   |   266 +
 .../src/test/bootstrap/aws/logs-collector.sh    |   173 +
 .../tests/ignite-cassandra-client-template.xml  |   192 +
 .../test/bootstrap/aws/tests/tests-bootstrap.sh |   317 +
 .../test/bootstrap/aws/tests/tests-manager.sh   |   458 +
 .../test/bootstrap/aws/tests/tests-report.sh    |   499 +
 .../CassandraDirectPersistenceLoadTest.java     |   107 +
 .../tests/CassandraDirectPersistenceTest.java   |   696 +
 .../ignite/tests/CassandraLocalServer.java      |    59 +
 .../apache/ignite/tests/DDLGeneratorTest.java   |    57 +
 .../tests/DatasourceSerializationTest.java      |   158 +
 .../tests/IgnitePersistentStoreLoadTest.java    |   111 +
 .../ignite/tests/IgnitePersistentStoreTest.java |   666 +
 .../LoadTestsCassandraArtifactsCreator.java     |   104 +
 .../org/apache/ignite/tests/load/Generator.java |    27 +
 .../apache/ignite/tests/load/IntGenerator.java  |    33 +
 .../ignite/tests/load/LoadTestDriver.java       |   238 +
 .../apache/ignite/tests/load/LongGenerator.java |    28 +
 .../ignite/tests/load/PersonGenerator.java      |    43 +
 .../ignite/tests/load/PersonIdGenerator.java    |    31 +
 .../ignite/tests/load/StringGenerator.java      |    28 +
 .../org/apache/ignite/tests/load/Worker.java    |   429 +
 .../tests/load/cassandra/BulkReadWorker.java    |    63 +
 .../tests/load/cassandra/BulkWriteWorker.java   |    52 +
 .../ignite/tests/load/cassandra/ReadWorker.java |    51 +
 .../tests/load/cassandra/WriteWorker.java       |    51 +
 .../tests/load/cassandra/package-info.java      |    21 +
 .../tests/load/ignite/BulkReadWorker.java       |    52 +
 .../tests/load/ignite/BulkWriteWorker.java      |    52 +
 .../ignite/tests/load/ignite/ReadWorker.java    |    51 +
 .../ignite/tests/load/ignite/WriteWorker.java   |    51 +
 .../ignite/tests/load/ignite/package-info.java  |    21 +
 .../apache/ignite/tests/load/package-info.java  |    21 +
 .../org/apache/ignite/tests/package-info.java   |    21 +
 .../org/apache/ignite/tests/pojos/Person.java   |   279 +
 .../org/apache/ignite/tests/pojos/PersonId.java |   110 +
 .../org/apache/ignite/tests/pojos/Product.java  |   123 +
 .../apache/ignite/tests/pojos/ProductOrder.java |   148 +
 .../apache/ignite/tests/pojos/package-info.java |    21 +
 .../ignite/tests/utils/CacheStoreHelper.java    |    77 +
 .../tests/utils/CassandraAdminCredentials.java  |    38 +
 .../ignite/tests/utils/CassandraHelper.java     |   358 +
 .../tests/utils/CassandraLifeCycleBean.java     |   149 +
 .../utils/CassandraRegularCredentials.java      |    38 +
 .../ignite/tests/utils/TestCacheSession.java    |    95 +
 .../ignite/tests/utils/TestTransaction.java     |   132 +
 .../apache/ignite/tests/utils/TestsHelper.java  |   660 +
 .../apache/ignite/tests/utils/package-info.java |    21 +
 .../store/src/test/resources/log4j.properties   |   119 +
 .../tests/cassandra/connection-settings.xml     |    52 +
 .../tests/cassandra/connection.properties       |    17 +
 .../tests/cassandra/credentials.properties      |    22 +
 .../tests/cassandra/embedded-cassandra.yaml     |   119 +
 .../ignite/tests/cassandra/keyspaces.properties |    17 +
 .../tests/persistence/blob/ignite-config.xml    |    94 +
 .../persistence/blob/persistence-settings-1.xml |    21 +
 .../persistence/blob/persistence-settings-2.xml |    21 +
 .../persistence/blob/persistence-settings-3.xml |    29 +
 .../tests/persistence/pojo/ignite-config.xml    |   169 +
 .../ignite/tests/persistence/pojo/order.xml     |    21 +
 .../persistence/pojo/persistence-settings-1.xml |    21 +
 .../persistence/pojo/persistence-settings-2.xml |    21 +
 .../persistence/pojo/persistence-settings-3.xml |   175 +
 .../persistence/pojo/persistence-settings-4.xml |   175 +
 .../ignite/tests/persistence/pojo/product.xml   |    21 +
 .../persistence/primitive/ignite-config.xml     |    94 +
 .../primitive/ignite-remote-client-config.xml   |    99 +
 .../primitive/ignite-remote-server-config.xml   |   110 +
 .../primitive/persistence-settings-1.xml        |    21 +
 .../primitive/persistence-settings-2.xml        |    21 +
 .../store/src/test/resources/tests.properties   |    65 +
 .../src/test/scripts/cassandra-load-tests.bat   |    41 +
 .../src/test/scripts/cassandra-load-tests.sh    |    39 +
 .../src/test/scripts/ignite-load-tests.bat      |    41 +
 .../store/src/test/scripts/ignite-load-tests.sh |    39 +
 .../cassandra/store/src/test/scripts/jvm-opt.sh |    21 +
 .../store/src/test/scripts/jvm-opts.bat         |    24 +
 .../scripts/recreate-cassandra-artifacts.bat    |    41 +
 .../scripts/recreate-cassandra-artifacts.sh     |    39 +
 modules/clients/pom.xml                         |    10 +-
 .../client/ClientDefaultCacheSelfTest.java      |   119 +-
 .../ignite/internal/client/ClientHttpTask.java  |    33 +-
 .../client/ClientReconnectionSelfTest.java      |     4 +-
 .../ignite/internal/client/ClientTcpTask.java   |    10 +-
 .../integration/ClientAbstractSelfTest.java     |    92 +-
 .../client/router/TcpSslRouterSelfTest.java     |     7 +-
 .../client/suite/IgniteClientTestSuite.java     |    71 +-
 .../jdbc2/JdbcDistributedJoinsQueryTest.java    |   319 +
 .../internal/jdbc2/JdbcMetadataSelfTest.java    |    63 +-
 .../internal/jdbc2/JdbcResultSetSelfTest.java   |    95 +-
 .../JettyRestProcessorAbstractSelfTest.java     |  1182 +-
 .../internal/processors/rest/SimplePerson.java  |    74 +
 .../ignite/jdbc/JdbcMetadataSelfTest.java       |    18 +
 .../ignite/jdbc/JdbcResultSetSelfTest.java      |     4 +-
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |     2 +
 modules/cloud/pom.xml                           |     2 +-
 .../TcpDiscoveryCloudIpFinderSelfTest.java      |     6 +-
 .../ignite/testsuites/IgniteCloudTestSuite.java |     5 +-
 modules/codegen/pom.xml                         |     2 +-
 .../ignite/codegen/MessageCodeGenerator.java    |   139 +-
 modules/core/pom.xml                            |    23 +-
 .../src/main/java/org/apache/ignite/Ignite.java |    77 +-
 .../java/org/apache/ignite/IgniteCache.java     |    40 +
 .../java/org/apache/ignite/IgniteCompute.java   |    69 +-
 .../org/apache/ignite/IgniteDataStreamer.java   |    57 +-
 .../IgniteDataStreamerTimeoutException.java     |    45 +
 .../org/apache/ignite/IgniteFileSystem.java     |    39 +-
 .../org/apache/ignite/IgniteJdbcDriver.java     |    14 +-
 .../java/org/apache/ignite/IgniteScheduler.java |    13 +
 .../java/org/apache/ignite/IgniteServices.java  |    48 +-
 .../apache/ignite/IgniteSystemProperties.java   |    42 +
 .../ignite/binary/BinaryObjectBuilder.java      |    11 +
 .../org/apache/ignite/cache/QueryEntity.java    |    18 +
 .../affinity/fair/FairAffinityFunction.java     |    91 +-
 .../rendezvous/RendezvousAffinityFunction.java  |    49 +-
 .../ignite/cache/query/ContinuousQuery.java     |    47 +-
 .../cache/query/QueryCancelledException.java    |    35 +
 .../apache/ignite/cache/query/QueryCursor.java  |     8 +-
 .../org/apache/ignite/cache/query/SpiQuery.java |     8 +-
 .../ignite/cache/query/SqlFieldsQuery.java      |    79 +
 .../org/apache/ignite/cache/query/SqlQuery.java |    53 +
 .../store/jdbc/CacheAbstractJdbcStore.java      |   339 +-
 .../cache/store/jdbc/CacheJdbcBlobStore.java    |     4 +-
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |    31 +-
 .../store/jdbc/CacheJdbcPojoStoreFactory.java   |    71 +-
 .../ignite/cache/store/jdbc/JdbcType.java       |     2 +-
 .../store/jdbc/JdbcTypesDefaultTransformer.java |   117 +
 .../cache/store/jdbc/JdbcTypesTransformer.java  |    39 +
 .../store/jdbc/dialect/BasicJdbcDialect.java    |    15 +-
 .../cache/store/jdbc/dialect/DB2Dialect.java    |     6 +-
 .../cache/store/jdbc/dialect/H2Dialect.java     |     5 +-
 .../cache/store/jdbc/dialect/JdbcDialect.java   |    29 +-
 .../cache/store/jdbc/dialect/MySQLDialect.java  |    27 +-
 .../cache/store/jdbc/dialect/OracleDialect.java |     5 +-
 .../store/jdbc/dialect/SQLServerDialect.java    |    16 +-
 .../apache/ignite/compute/ComputeTaskSpis.java  |     8 +-
 .../configuration/BasicAddressResolver.java     |   146 +
 .../configuration/CacheConfiguration.java       |    54 +-
 .../configuration/ConnectorConfiguration.java   |     1 +
 .../configuration/FileSystemConfiguration.java  |    56 +-
 .../configuration/HadoopConfiguration.java      |    39 +-
 .../configuration/IgniteConfiguration.java      |    60 +-
 .../ignite/configuration/OdbcConfiguration.java |    98 +-
 .../java/org/apache/ignite/igfs/IgfsPath.java   |    40 +-
 .../org/apache/ignite/igfs/IgfsPathSummary.java |    35 +-
 .../igfs/secondary/IgfsSecondaryFileSystem.java |     6 +-
 .../local/LocalIgfsSecondaryFileSystem.java     |   485 +
 .../igfs/secondary/local/package-info.java      |    22 +
 .../ignite/igfs/secondary/package-info.java     |     2 +-
 .../ignite/internal/GridCodegenConverter.java   |    56 +
 .../internal/GridEventConsumeHandler.java       |    18 +-
 .../ignite/internal/GridJobExecuteRequest.java  |   148 +-
 .../ignite/internal/GridJobExecuteResponse.java |    42 +-
 .../internal/GridJobSiblingsResponse.java       |     3 +-
 .../ignite/internal/GridKernalContext.java      |    31 +
 .../ignite/internal/GridKernalContextImpl.java  |    52 +-
 .../apache/ignite/internal/GridLoggerProxy.java |     3 +-
 .../internal/GridMessageListenHandler.java      |    18 +-
 .../ignite/internal/GridTaskSessionImpl.java    |    15 +-
 .../ignite/internal/IgniteComponentType.java    |    11 +-
 .../ignite/internal/IgniteComputeImpl.java      |   120 +-
 .../apache/ignite/internal/IgniteKernal.java    |   181 +-
 .../ignite/internal/IgniteNodeAttributes.java   |     3 +
 .../ignite/internal/IgniteSchedulerImpl.java    |    18 +
 .../ignite/internal/IgniteServicesImpl.java     |     6 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   194 +-
 .../apache/ignite/internal/LessNamingBean.java  |    28 +
 .../ignite/internal/MarshallerContextImpl.java  |   101 +-
 .../internal/binary/BinaryClassDescriptor.java  |    59 +-
 .../ignite/internal/binary/BinaryContext.java   |   165 +-
 .../internal/binary/BinaryEnumObjectImpl.java   |    17 +-
 .../internal/binary/BinaryMarshaller.java       |    30 +-
 .../binary/BinaryMethodWriteReplacer.java       |    59 +
 .../ignite/internal/binary/BinaryObjectEx.java  |    19 +
 .../internal/binary/BinaryObjectExImpl.java     |   166 +-
 .../internal/binary/BinaryObjectImpl.java       |    34 +-
 .../binary/BinaryObjectOffheapImpl.java         |    15 +-
 .../internal/binary/BinaryReaderExImpl.java     |    24 +
 .../internal/binary/BinaryReaderHandles.java    |     2 +-
 .../ignite/internal/binary/BinaryTreeMap.java   |    96 +
 .../binary/BinaryTreeMapWriteReplacer.java      |    34 +
 .../ignite/internal/binary/BinaryTreeSet.java   |    93 +
 .../binary/BinaryTreeSetWriteReplacer.java      |    34 +
 .../ignite/internal/binary/BinaryTypeProxy.java |   111 +
 .../ignite/internal/binary/BinaryUtils.java     |    93 +-
 .../internal/binary/BinaryWriteReplacer.java    |    33 +
 .../internal/binary/BinaryWriterExImpl.java     |    47 +-
 .../binary/builder/BinaryObjectBuilderImpl.java |    13 +-
 .../client/GridClientConfiguration.java         |     1 -
 .../GridClientConnectionManagerAdapter.java     |    25 +-
 .../connection/GridClientNioTcpConnection.java  |     3 +
 .../GridClientOptimizedMarshaller.java          |     9 +-
 .../GridClientZipOptimizedMarshaller.java       |   168 +
 .../impl/GridTcpRouterNioListenerAdapter.java   |    11 +-
 .../internal/cluster/ClusterGroupAdapter.java   |    26 +-
 .../ignite/internal/cluster/ClusterGroupEx.java |    14 +-
 .../cluster/ClusterNodeLocalMapImpl.java        |     3 +-
 .../internal/cluster/IgniteClusterImpl.java     |     9 +-
 .../igfs/common/IgfsControlResponse.java        |    19 +-
 .../internal/igfs/common/IgfsMarshaller.java    |    14 +-
 .../apache/ignite/internal/jdbc/JdbcUtils.java  |     4 +-
 .../ignite/internal/jdbc2/JdbcConnection.java   |    12 +
 .../ignite/internal/jdbc2/JdbcQueryTask.java    |     9 +-
 .../ignite/internal/jdbc2/JdbcResultSet.java    |     6 +-
 .../ignite/internal/jdbc2/JdbcStatement.java    |     2 +-
 .../logger/platform/PlatformLogger.java         |   223 +
 .../internal/managers/GridManagerAdapter.java   |    15 +
 .../checkpoint/GridCheckpointManager.java       |     6 +-
 .../managers/communication/GridIoManager.java   |   188 +-
 .../communication/GridIoMessageFactory.java     |     8 +-
 .../managers/communication/GridIoPolicy.java    |     5 +-
 .../deployment/GridDeploymentCommunication.java |     4 +-
 .../GridDeploymentPerVersionStore.java          |     2 +-
 .../discovery/GridDiscoveryManager.java         |   132 +-
 .../eventstorage/GridEventStorageManager.java   |    18 +-
 .../failover/GridFailoverContextImpl.java       |    27 +-
 .../managers/failover/GridFailoverManager.java  |    17 +-
 .../loadbalancer/GridLoadBalancerManager.java   |    10 +-
 .../swapspace/GridSwapSpaceManager.java         |    13 +-
 .../GridAffinityFunctionContextImpl.java        |     9 +
 .../affinity/GridAffinityProcessor.java         |   135 +-
 .../processors/affinity/GridAffinityUtils.java  |     4 +-
 .../cache/CacheEntrySerializablePredicate.java  |     4 +-
 .../cache/CacheEvictableEntryImpl.java          |    11 +-
 .../cache/CacheInvokeDirectResult.java          |     4 +-
 .../processors/cache/CacheLazyEntry.java        |     2 +
 .../processors/cache/CacheObjectContext.java    |    15 +
 .../EntryProcessorResourceInjectorProxy.java    |   105 +
 .../processors/cache/GridCacheAdapter.java      |   404 +-
 .../cache/GridCacheAffinityManager.java         |    26 +-
 .../processors/cache/GridCacheContext.java      |    62 +-
 .../GridCacheDefaultAffinityKeyMapper.java      |    19 +
 .../cache/GridCacheDeploymentManager.java       |     4 +-
 .../processors/cache/GridCacheEntryEx.java      |    20 +-
 .../processors/cache/GridCacheEntryInfo.java    |     2 +-
 .../processors/cache/GridCacheEventManager.java |    10 +-
 .../cache/GridCacheEvictionManager.java         |     2 +-
 .../processors/cache/GridCacheIoManager.java    |   172 +-
 .../processors/cache/GridCacheMapEntry.java     |   426 +-
 .../processors/cache/GridCacheMessage.java      |    15 +-
 .../GridCachePartitionExchangeManager.java      |   153 +-
 .../processors/cache/GridCacheProcessor.java    |   344 +-
 .../processors/cache/GridCacheProxyImpl.java    |    36 +
 .../GridCacheReturnCompletableWrapper.java      |   101 +
 .../cache/GridCacheSharedContext.java           |    92 +-
 .../cache/GridCacheSharedTtlCleanupManager.java |   132 +
 .../processors/cache/GridCacheSwapManager.java  |   112 +-
 .../processors/cache/GridCacheTtlManager.java   |   120 +-
 .../processors/cache/GridCacheUtils.java        |    31 +-
 .../cache/GridDeferredAckMessageSender.java     |   219 +
 .../processors/cache/IgniteCacheProxy.java      |   236 +-
 .../processors/cache/IgniteInternalCache.java   |    23 +
 .../processors/cache/KeyCacheObject.java        |     7 +
 .../processors/cache/KeyCacheObjectImpl.java    |     8 +
 .../processors/cache/QueryCursorImpl.java       |    92 +-
 .../cache/binary/CacheObjectBinaryContext.java  |    10 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |   133 +-
 .../distributed/GridCacheTxRecoveryFuture.java  |   134 +-
 .../distributed/GridCacheTxRecoveryRequest.java |     7 +
 .../GridCacheTxRecoveryResponse.java            |     7 +
 .../distributed/GridDistributedLockRequest.java |     6 +
 .../GridDistributedLockResponse.java            |    10 +-
 .../GridDistributedTxFinishRequest.java         |     7 +
 .../GridDistributedTxFinishResponse.java        |     7 +
 .../GridDistributedTxPrepareRequest.java        |    14 +-
 .../GridDistributedTxPrepareResponse.java       |    11 +-
 .../GridDistributedTxRemoteAdapter.java         |    65 +-
 .../GridDistributedUnlockRequest.java           |     6 +
 .../dht/GridCachePartitionedConcurrentMap.java  |   142 +-
 .../dht/GridDhtAffinityAssignmentRequest.java   |     2 +-
 .../dht/GridDhtAffinityAssignmentResponse.java  |    12 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |    47 +-
 .../distributed/dht/GridDhtCacheEntry.java      |     8 +
 .../cache/distributed/dht/GridDhtGetFuture.java |    23 +-
 .../distributed/dht/GridDhtLocalPartition.java  |    40 +-
 .../distributed/dht/GridDhtLockFuture.java      |   134 +-
 .../dht/GridDhtOffHeapCacheEntry.java           |     8 +
 .../dht/GridDhtPartitionTopology.java           |     2 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   219 +-
 .../dht/GridDhtPartitionsReservation.java       |     3 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |    79 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |   108 +-
 .../distributed/dht/GridDhtTxFinishRequest.java |    33 +-
 .../dht/GridDhtTxFinishResponse.java            |    56 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |    53 +-
 .../dht/GridDhtTxOnePhaseCommitAckRequest.java  |   134 +
 .../distributed/dht/GridDhtTxPrepareFuture.java |   262 +-
 .../dht/GridDhtTxPrepareRequest.java            |    97 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |     6 +-
 .../dht/GridPartitionedGetFuture.java           |     2 -
 .../dht/GridPartitionedSingleGetFuture.java     |     2 -
 .../dht/atomic/GridDhtAtomicCache.java          |   525 +-
 .../GridDhtAtomicDeferredUpdateResponse.java    |     7 +
 .../atomic/GridDhtAtomicOffHeapCacheEntry.java  |     8 +
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |    47 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |     6 +
 .../dht/atomic/GridDhtAtomicUpdateResponse.java |    10 +-
 .../GridNearAtomicAbstractUpdateFuture.java     |    76 +-
 .../GridNearAtomicSingleUpdateFuture.java       |    91 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |    99 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |     8 +-
 .../atomic/GridNearAtomicUpdateResponse.java    |    10 +-
 .../dht/colocated/GridDhtColocatedCache.java    |    14 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   114 +-
 .../GridDhtColocatedOffHeapCacheEntry.java      |     8 +
 .../colocated/GridDhtDetachedCacheEntry.java    |     7 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |     3 +-
 .../dht/preloader/GridDhtForceKeysResponse.java |     4 +-
 .../GridDhtPartitionDemandMessage.java          |     4 +-
 .../dht/preloader/GridDhtPartitionDemander.java |     3 +-
 .../dht/preloader/GridDhtPartitionMap2.java     |     7 +-
 .../GridDhtPartitionsExchangeFuture.java        |    45 +-
 .../preloader/GridDhtPartitionsFullMessage.java |     8 +-
 .../GridDhtPartitionsSingleMessage.java         |     8 +-
 .../distributed/near/GridNearCacheAdapter.java  |     7 +
 .../distributed/near/GridNearGetFuture.java     |    12 +-
 .../distributed/near/GridNearGetResponse.java   |     4 +-
 .../distributed/near/GridNearLockFuture.java    |    96 +-
 ...arOptimisticSerializableTxPrepareFuture.java |    23 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   293 +-
 ...ridNearOptimisticTxPrepareFutureAdapter.java |     5 +-
 .../GridNearPessimisticTxPrepareFuture.java     |    54 +-
 .../near/GridNearSingleGetResponse.java         |     4 +-
 .../near/GridNearTransactionalCache.java        |     2 +-
 .../near/GridNearTxFinishFuture.java            |   242 +-
 .../near/GridNearTxFinishResponse.java          |     4 +-
 .../cache/distributed/near/GridNearTxLocal.java |    39 +-
 .../near/GridNearTxPrepareFutureAdapter.java    |     7 +-
 .../near/GridNearTxPrepareRequest.java          |     4 +-
 .../local/atomic/GridLocalAtomicCache.java      |    26 +-
 .../processors/cache/query/CacheQuery.java      |    11 +-
 .../query/GridCacheDistributedQueryManager.java |    22 +-
 .../cache/query/GridCacheLocalQueryFuture.java  |     4 +-
 .../cache/query/GridCacheLocalQueryManager.java |     3 +-
 .../cache/query/GridCacheQueryAdapter.java      |    69 +-
 .../cache/query/GridCacheQueryBean.java         |     8 +-
 .../cache/query/GridCacheQueryInfo.java         |     8 +-
 .../cache/query/GridCacheQueryManager.java      |   259 +-
 .../cache/query/GridCacheQueryMarshallable.java |    37 +
 .../query/GridCacheQueryMetricsAdapter.java     |    12 +-
 .../cache/query/GridCacheQueryRequest.java      |    18 +-
 .../cache/query/GridCacheQueryResponse.java     |     4 +-
 .../cache/query/GridCacheSqlIndexMetadata.java  |     3 +-
 .../cache/query/GridCacheSqlMetadata.java       |     3 +-
 .../cache/query/GridCacheSqlQuery.java          |    42 +-
 .../cache/query/GridCacheTwoStepQuery.java      |   123 +-
 .../continuous/CacheContinuousQueryEntry.java   |    54 +-
 .../continuous/CacheContinuousQueryHandler.java |   102 +-
 .../continuous/CacheContinuousQueryManager.java |   148 +-
 .../jdbc/GridCacheQueryJdbcMetadataTask.java    |     4 +-
 .../query/jdbc/GridCacheQueryJdbcTask.java      |     6 +-
 .../store/GridCacheStoreManagerAdapter.java     |     2 +-
 .../cache/store/GridCacheWriteBehindStore.java  |     2 +-
 .../cache/transactions/IgniteInternalTx.java    |     3 +-
 .../cache/transactions/IgniteTxAdapter.java     |   100 +-
 .../cache/transactions/IgniteTxEntry.java       |    71 +-
 .../cache/transactions/IgniteTxHandler.java     |   463 +-
 .../transactions/IgniteTxLocalAdapter.java      |   159 +-
 .../cache/transactions/IgniteTxManager.java     |   272 +-
 .../cache/transactions/IgniteTxStateImpl.java   |    11 +-
 .../cache/transactions/TxDeadlockDetection.java |    51 +-
 .../GridCacheLazyPlainVersionedEntry.java       |   113 +
 .../version/GridCachePlainVersionedEntry.java   |    12 +-
 .../version/GridCacheRawVersionedEntry.java     |     8 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |    28 +-
 .../IgniteCacheObjectProcessorImpl.java         |    82 +-
 .../clock/GridClockSyncProcessor.java           |     2 +-
 .../processors/closure/AffinityTask.java        |    17 +-
 .../processors/closure/GridClosurePolicy.java   |    51 -
 .../closure/GridClosureProcessor.java           |   239 +-
 .../processors/cluster/ClusterProcessor.java    |     2 +-
 .../processors/cluster/GridUpdateNotifier.java  |   108 +-
 .../continuous/GridContinuousHandler.java       |    13 +-
 .../continuous/GridContinuousProcessor.java     |   264 +-
 .../continuous/GridContinuousQueryBatch.java    |    47 +
 .../processors/continuous/StartRequestData.java |     4 +-
 .../datastreamer/DataStreamProcessor.java       |     8 +-
 .../datastreamer/DataStreamerImpl.java          |   156 +-
 .../GridCacheCountDownLatchImpl.java            |    54 +-
 .../datastructures/GridCacheQueueAdapter.java   |     2 +-
 .../datastructures/GridCacheSemaphoreImpl.java  |   108 +-
 .../processors/hadoop/HadoopClassLoader.java    |   487 +
 .../processors/hadoop/HadoopClasspathUtils.java |   424 +
 .../processors/hadoop/HadoopDefaultJobInfo.java |   156 +
 .../processors/hadoop/HadoopHelper.java         |    62 +
 .../internal/processors/hadoop/HadoopJob.java   |     5 +
 .../processors/hadoop/HadoopJobInfo.java        |     5 +-
 .../processors/hadoop/HadoopLocations.java      |   123 +
 .../hadoop/HadoopMapReducePlanner.java          |     1 +
 .../processors/hadoop/HadoopNoopHelper.java     |    71 +
 .../processors/hadoop/HadoopNoopProcessor.java  |     9 +-
 .../hadoop/HadoopProcessorAdapter.java          |     7 +
 .../igfs/IgfsAbstractOutputStream.java          |   266 +
 .../processors/igfs/IgfsAckMessage.java         |     5 +-
 .../internal/processors/igfs/IgfsAsyncImpl.java |    29 +-
 .../processors/igfs/IgfsBlockLocationImpl.java  |    87 +-
 .../internal/processors/igfs/IgfsContext.java   |    70 +-
 .../processors/igfs/IgfsCreateResult.java       |    66 +
 .../processors/igfs/IgfsDataManager.java        |   393 +-
 .../processors/igfs/IgfsDeleteMessage.java      |     5 +-
 .../processors/igfs/IgfsDeleteResult.java       |    62 +
 .../processors/igfs/IgfsDeleteWorker.java       |    36 -
 .../ignite/internal/processors/igfs/IgfsEx.java |    34 +-
 .../processors/igfs/IgfsFileAffinityRange.java  |    40 +-
 .../internal/processors/igfs/IgfsFileImpl.java  |    59 +-
 .../igfs/IgfsFragmentizerManager.java           |    29 +-
 .../internal/processors/igfs/IgfsImpl.java      |  1174 +-
 .../processors/igfs/IgfsInputStreamAdapter.java |    51 -
 .../processors/igfs/IgfsInputStreamImpl.java    |   219 +-
 .../processors/igfs/IgfsIpcHandler.java         |     7 +-
 .../processors/igfs/IgfsKernalContextAware.java |    32 +
 ...zySecondaryFileSystemPositionedReadable.java |    77 +
 .../processors/igfs/IgfsMetaManager.java        |  1040 +-
 .../processors/igfs/IgfsModeResolver.java       |   104 +-
 .../processors/igfs/IgfsNodePredicate.java      |    80 +
 .../igfs/IgfsOutputStreamAdapter.java           |   265 -
 .../processors/igfs/IgfsOutputStreamImpl.java   |   406 +-
 .../igfs/IgfsOutputStreamProxyImpl.java         |   163 +
 .../internal/processors/igfs/IgfsPathIds.java   |    12 +-
 .../internal/processors/igfs/IgfsPaths.java     |    15 +-
 .../internal/processors/igfs/IgfsProcessor.java |    80 +-
 .../IgfsSecondaryFileSystemCreateContext.java   |   114 +
 .../igfs/IgfsSecondaryFileSystemImpl.java       |    17 +-
 .../igfs/IgfsSecondaryFileSystemV2.java         |    40 +
 .../IgfsSecondaryOutputStreamDescriptor.java    |    59 -
 .../internal/processors/igfs/IgfsServer.java    |     5 +-
 .../processors/igfs/IgfsThreadFactory.java      |    61 +
 .../internal/processors/igfs/IgfsUtils.java     |   258 +-
 .../igfs/client/IgfsClientAbstractCallable.java |   125 +
 .../igfs/client/IgfsClientAffinityCallable.java |    95 +
 .../igfs/client/IgfsClientDeleteCallable.java   |    77 +
 .../igfs/client/IgfsClientExistsCallable.java   |    58 +
 .../igfs/client/IgfsClientInfoCallable.java     |    59 +
 .../client/IgfsClientListFilesCallable.java     |    61 +
 .../client/IgfsClientListPathsCallable.java     |    60 +
 .../igfs/client/IgfsClientMkdirsCallable.java   |    82 +
 .../igfs/client/IgfsClientRenameCallable.java   |    80 +
 .../igfs/client/IgfsClientSetTimesCallable.java |    87 +
 .../igfs/client/IgfsClientSizeCallable.java     |    59 +
 .../igfs/client/IgfsClientSummaryCallable.java  |    59 +
 .../igfs/client/IgfsClientUpdateCallable.java   |    81 +
 .../meta/IgfsClientMetaIdsForPathCallable.java  |    65 +
 .../meta/IgfsClientMetaInfoForPathCallable.java |    63 +
 .../meta/IgfsClientMetaUnlockCallable.java      |   123 +
 .../igfs/data/IgfsDataPutProcessor.java         |    99 +
 .../meta/IgfsMetaDirectoryCreateProcessor.java  |    40 +-
 ...IgfsMetaDirectoryListingRenameProcessor.java |   133 +
 .../igfs/meta/IgfsMetaFileCreateProcessor.java  |    46 +-
 .../igfs/meta/IgfsMetaFileUnlockProcessor.java  |    69 +-
 .../local/LocalFileSystemIgfsFile.java          |   133 +
 .../local/LocalFileSystemSizeVisitor.java       |    60 +
 .../secondary/local/LocalFileSystemUtils.java   |   142 +
 ...fsSecondaryFileSystemPositionedReadable.java |    65 +
 .../processors/job/GridJobProcessor.java        |   160 +-
 .../internal/processors/job/GridJobWorker.java  |   219 +-
 .../OsDiscoveryNodeValidationProcessor.java     |     2 +-
 .../processors/odbc/OdbcHandshakeRequest.java   |    42 +-
 .../processors/odbc/OdbcHandshakeResult.java    |    17 +-
 .../processors/odbc/OdbcMessageParser.java      |    44 +-
 .../processors/odbc/OdbcNioListener.java        |     2 +-
 .../internal/processors/odbc/OdbcProcessor.java |    54 +-
 .../processors/odbc/OdbcProtocolVersion.java    |   125 +
 .../processors/odbc/OdbcRequestHandler.java     |    85 +-
 .../odbc/escape/OdbcEscapeParseResult.java      |    73 +
 .../processors/odbc/escape/OdbcEscapeToken.java |    61 +
 .../processors/odbc/escape/OdbcEscapeType.java  |   112 +
 .../processors/odbc/escape/OdbcEscapeUtils.java |   392 +
 .../offheap/GridOffHeapProcessor.java           |    30 +-
 .../platform/PlatformAbstractBootstrap.java     |    23 +-
 .../platform/PlatformAbstractTarget.java        |   199 +-
 .../processors/platform/PlatformBootstrap.java  |     6 +
 .../platform/PlatformConfigurationEx.java       |    14 +
 .../platform/PlatformContextImpl.java           |     3 -
 .../processors/platform/PlatformIgnition.java   |     7 +-
 .../platform/PlatformNoopProcessor.java         |    15 +
 .../processors/platform/PlatformProcessor.java  |    33 +-
 .../platform/PlatformProcessorImpl.java         |   184 +-
 .../processors/platform/PlatformTarget.java     |    44 +-
 .../binary/PlatformBinaryProcessor.java         |    96 +
 .../platform/cache/PlatformCache.java           |   972 +-
 .../platform/cache/PlatformCacheExtension.java  |    47 +
 .../cache/affinity/PlatformAffinity.java        |    14 +-
 .../affinity/PlatformAffinityFunction.java      |   309 +
 .../PlatformAffinityFunctionTarget.java         |   113 +
 .../cache/affinity/PlatformAffinityUtils.java   |   118 +
 .../query/PlatformAbstractQueryCursor.java      |    53 +-
 .../query/PlatformContinuousQueryProxy.java     |    54 +
 .../cache/query/PlatformFieldsQueryCursor.java  |     6 +
 .../cache/store/PlatformCacheStoreCallback.java |    61 -
 .../callback/PlatformCallbackGateway.java       |   152 +-
 .../callback/PlatformCallbackUtils.java         |    80 +-
 .../platform/cluster/PlatformClusterGroup.java  |   146 +-
 .../platform/compute/PlatformAbstractTask.java  |     1 +
 .../platform/compute/PlatformCompute.java       |    99 +-
 .../cpp/PlatformCppConfigurationClosure.java    |    13 +-
 .../cpp/PlatformCppConfigurationEx.java         |    13 +
 .../datastreamer/PlatformDataStreamer.java      |   105 +-
 .../datastructures/PlatformAtomicLong.java      |   174 +-
 .../datastructures/PlatformAtomicReference.java |    38 +-
 .../datastructures/PlatformAtomicSequence.java  |   126 +-
 .../dotnet/PlatformDotNetBootstrap.java         |    21 +
 .../dotnet/PlatformDotNetCacheStore.java        |   146 +-
 .../PlatformDotNetConfigurationClosure.java     |    67 +-
 .../dotnet/PlatformDotNetConfigurationEx.java   |    27 +-
 .../dotnet/PlatformDotNetConsoleStream.java     |    54 +
 ...PlatformDotNetEntityFrameworkCacheEntry.java |   102 +
 ...formDotNetEntityFrameworkCacheExtension.java |   353 +
 .../PlatformDotNetEntityFrameworkCacheKey.java  |   164 +
 ...EntityFrameworkIncreaseVersionProcessor.java |    45 +
 .../platform/events/PlatformEvents.java         |   194 +-
 .../platform/messaging/PlatformMessaging.java   |    83 +-
 .../services/PlatformAbstractService.java       |     3 +-
 .../platform/services/PlatformServices.java     |   232 +-
 .../transactions/PlatformTransactions.java      |   220 +-
 .../utils/PlatformConfigurationUtils.java       |   226 +-
 .../platform/utils/PlatformFutureUtils.java     |     4 +-
 .../platform/utils/PlatformUtils.java           |    45 +-
 .../PlatformDotNetSessionCacheExtension.java    |   144 +
 .../websession/PlatformDotNetSessionData.java   |   260 +
 .../PlatformDotNetSessionLockProcessor.java     |    84 +
 .../PlatformDotNetSessionLockResult.java        |   106 +
 ...tformDotNetSessionSetAndUnlockProcessor.java |   179 +
 .../internal/processors/pool/PoolProcessor.java |   154 +
 .../processors/query/GridQueryCancel.java       |    83 +
 .../processors/query/GridQueryFieldsResult.java |     3 +-
 .../query/GridQueryFieldsResultAdapter.java     |     3 +-
 .../processors/query/GridQueryIndexing.java     |    60 +-
 .../processors/query/GridQueryProcessor.java    |   317 +-
 .../query/GridQueryTypeDescriptor.java          |     7 +
 .../messages/GridQueryCancelRequest.java        |     2 +-
 .../twostep/messages/GridQueryFailResponse.java |    34 +-
 .../messages/GridQueryNextPageRequest.java      |     2 +-
 .../messages/GridQueryNextPageResponse.java     |    12 +-
 .../h2/twostep/messages/GridQueryRequest.java   |    59 +-
 .../processors/resource/GridResourceIoc.java    |   438 +-
 .../resource/GridResourceProcessor.java         |   396 +-
 .../message/GridClientHandshakeRequest.java     |     4 +-
 .../handlers/cache/GridCacheCommandHandler.java |   100 +-
 .../handlers/query/QueryCommandHandler.java     |    20 +-
 .../handlers/task/GridTaskCommandHandler.java   |    10 +-
 .../protocols/tcp/GridTcpRestNioListener.java   |    19 +-
 .../rest/protocols/tcp/GridTcpRestParser.java   |     4 +-
 .../rest/protocols/tcp/GridTcpRestProtocol.java |    12 +-
 .../rest/request/RestQueryRequest.java          |    19 +-
 .../service/GridServiceAssignments.java         |    10 +-
 .../service/GridServiceProcessor.java           |   632 +-
 .../service/LazyServiceConfiguration.java       |   129 +
 .../service/ServiceDescriptorImpl.java          |    17 +-
 .../session/GridTaskSessionProcessor.java       |     9 +-
 .../processors/task/GridTaskProcessor.java      |    11 +-
 .../processors/task/GridTaskWorker.java         |   279 +-
 .../apache/ignite/internal/util/ClassCache.java |    32 +
 .../util/GridBoundedConcurrentOrderedMap.java   |     5 +
 .../ignite/internal/util/GridJavaProcess.java   |     5 +-
 .../ignite/internal/util/GridLogThrottle.java   |    35 +-
 .../internal/util/IgniteExceptionRegistry.java  |     5 +-
 .../ignite/internal/util/IgniteUtils.java       |   557 +-
 .../internal/util/PartitionedReadOnlySet.java   |    71 -
 .../internal/util/SerializableTransient.java    |    58 +
 .../util/future/GridCompoundFuture.java         |   100 +-
 .../ipc/shmem/IpcSharedMemoryNativeLoader.java  |     2 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |    19 +-
 .../ignite/internal/util/lang/GridFunc.java     |    20 +
 .../nio/GridConnectionBytesVerifyFilter.java    |     2 +-
 .../internal/util/nio/GridNioCodecFilter.java   |     2 +-
 .../internal/util/nio/GridNioFilterChain.java   |     2 +-
 .../util/nio/GridNioRecoveryDescriptor.java     |    19 +-
 .../ignite/internal/util/nio/GridNioServer.java |    59 +-
 .../util/nio/GridNioSessionMetaKey.java         |     7 +-
 .../util/nio/GridSelectorNioSessionImpl.java    |     9 +-
 .../util/nio/GridTcpNioCommunicationClient.java |     5 +-
 .../util/nio/ssl/BlockingSslHandler.java        |    61 +-
 .../internal/util/nio/ssl/GridNioSslFilter.java |    63 +-
 .../util/nio/ssl/GridNioSslHandler.java         |    17 +-
 .../internal/util/nio/ssl/GridSslMeta.java      |    94 +
 .../offheap/unsafe/GridOffHeapSnapTreeMap.java  |    91 +-
 .../util/offheap/unsafe/GridUnsafeLru.java      |    30 +-
 .../ignite/internal/visor/cache/VisorCache.java |    60 +-
 .../cache/VisorCacheAffinityConfiguration.java  |     5 +-
 .../cache/VisorCacheAggregatedMetrics.java      |     3 +-
 .../visor/cache/VisorCacheConfiguration.java    |     3 +-
 .../cache/VisorCacheDefaultConfiguration.java   |     5 +-
 .../cache/VisorCacheEvictionConfiguration.java  |     5 +-
 .../internal/visor/cache/VisorCacheMetrics.java |     3 +-
 .../cache/VisorCacheNearConfiguration.java      |     5 +-
 .../visor/cache/VisorCachePartition.java        |    90 +
 .../visor/cache/VisorCachePartitions.java       |    89 +
 .../visor/cache/VisorCachePartitionsTask.java   |   152 +
 .../cache/VisorCacheQueryConfiguration.java     |     3 +-
 .../visor/cache/VisorCacheQueryMetrics.java     |     5 +-
 .../cache/VisorCacheRebalanceConfiguration.java |     5 +-
 .../cache/VisorCacheResetQueryMetricsTask.java  |    69 +
 .../cache/VisorCacheStoreConfiguration.java     |     3 +-
 .../cache/VisorCacheTypeFieldMetadata.java      |     3 +-
 .../visor/cache/VisorCacheTypeMetadata.java     |    69 +-
 .../internal/visor/cache/VisorCacheV3.java      |    68 +-
 .../internal/visor/cache/VisorCacheV4.java      |   124 +
 .../visor/compute/VisorGatewayTask.java         |     5 +-
 .../internal/visor/debug/VisorThreadInfo.java   |     5 +-
 .../visor/debug/VisorThreadLockInfo.java        |     5 +-
 .../visor/event/VisorGridDiscoveryEventV2.java  |    80 +
 .../internal/visor/event/VisorGridEvent.java    |     5 +-
 .../internal/visor/file/VisorFileBlock.java     |     5 +-
 .../ignite/internal/visor/igfs/VisorIgfs.java   |     7 +-
 .../internal/visor/igfs/VisorIgfsEndpoint.java  |     5 +-
 .../internal/visor/igfs/VisorIgfsMetrics.java   |    16 +-
 .../visor/igfs/VisorIgfsProfilerEntry.java      |     5 +-
 .../VisorIgfsProfilerUniformityCounters.java    |     5 +-
 .../visor/log/VisorLogSearchResult.java         |     5 +-
 .../visor/node/VisorAtomicConfiguration.java    |     5 +-
 .../visor/node/VisorBasicConfiguration.java     |     5 +-
 .../node/VisorExecutorServiceConfiguration.java |     5 +-
 .../visor/node/VisorGridConfiguration.java      |     5 +-
 .../visor/node/VisorIgfsConfiguration.java      |     3 +-
 .../visor/node/VisorLifecycleConfiguration.java |     5 +-
 .../visor/node/VisorMetricsConfiguration.java   |     5 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |    63 +-
 .../visor/node/VisorNodeDataCollectorTask.java  |    26 -
 .../node/VisorNodeDataCollectorTaskResult.java  |     5 +-
 .../node/VisorPeerToPeerConfiguration.java      |     5 +-
 .../visor/node/VisorRestConfiguration.java      |     5 +-
 .../node/VisorSegmentationConfiguration.java    |     5 +-
 .../visor/node/VisorSpisConfiguration.java      |     5 +-
 .../node/VisorTransactionConfiguration.java     |     5 +-
 .../internal/visor/query/VisorQueryArgV2.java   |    49 +
 .../internal/visor/query/VisorQueryField.java   |     5 +-
 .../internal/visor/query/VisorQueryJob.java     |    39 +-
 .../internal/visor/query/VisorQueryResult.java  |     5 +-
 .../query/VisorQueryScanSubstringFilter.java    |    63 +
 .../internal/visor/query/VisorQueryUtils.java   |     6 +
 .../internal/visor/util/VisorEventMapper.java   |    96 +-
 .../internal/visor/util/VisorTaskUtils.java     |    30 +-
 .../apache/ignite/logger/java/JavaLogger.java   |    14 +-
 .../logger/java/JavaLoggerFileHandler.java      |    10 +-
 .../ignite/marshaller/AbstractMarshaller.java   |    41 +-
 .../AbstractNodeNameAwareMarshaller.java        |   142 +
 .../ignite/marshaller/MarshallerUtils.java      |    80 +
 .../ignite/marshaller/jdk/JdkMarshaller.java    |    48 +-
 .../optimized/OptimizedClassDescriptor.java     |    90 +-
 .../optimized/OptimizedMarshaller.java          |    26 +-
 .../dotnet/PlatformDotNetAffinityFunction.java  |   181 +
 .../plugin/security/SecurityPermissionSet.java  |     5 +-
 .../ignite/plugin/security/SecuritySubject.java |     5 +-
 .../SpringApplicationContextResource.java       |     4 +-
 .../apache/ignite/resources/SpringResource.java |    15 +-
 .../ignite/scheduler/SchedulerFuture.java       |     5 +-
 .../org/apache/ignite/services/Service.java     |    12 +-
 .../ignite/services/ServiceConfiguration.java   |    14 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |    11 +
 .../org/apache/ignite/spi/IgniteSpiContext.java |    26 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |    10 +-
 .../spi/checkpoint/sharedfs/SharedFsUtils.java  |     4 +-
 .../communication/tcp/TcpCommunicationSpi.java  |   239 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |    51 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   260 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |     5 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |    35 +-
 .../tcp/internal/TcpDiscoveryStatistics.java    |    45 +-
 .../ipfinder/jdbc/BasicJdbcIpFinderDialect.java |    28 +
 .../tcp/ipfinder/jdbc/JdbcIpFinderDialect.java  |    28 +
 .../jdbc/OracleJdbcIpFinderDialect.java         |    28 +
 .../ipfinder/jdbc/TcpDiscoveryJdbcIpFinder.java |    72 +-
 .../TcpDiscoveryMulticastIpFinder.java          |    11 +-
 .../sharedfs/TcpDiscoverySharedFsIpFinder.java  |    62 +-
 .../TcpDiscoveryCustomEventMessage.java         |     3 +-
 .../ignite/spi/failover/FailoverContext.java    |    15 +-
 .../spi/failover/always/AlwaysFailoverSpi.java  |    15 +-
 .../RoundRobinGlobalLoadBalancer.java           |    16 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |    95 +-
 .../ignite/stream/socket/SocketStreamer.java    |    19 +-
 .../java/org/jetbrains/annotations/NotNull.java |    38 -
 .../org/jetbrains/annotations/Nullable.java     |    33 -
 .../org/jetbrains/annotations/package-info.java |    27 -
 .../resources/META-INF/classnames.properties    |   181 +-
 .../core/src/main/resources/ignite.properties   |     2 +-
 modules/core/src/test/config/log4j-test.xml     |     6 +
 .../AbstractAffinityFunctionSelfTest.java       |    18 +-
 .../affinity/AffinityClientNodeSelfTest.java    |    73 +-
 ...ityFunctionBackupFilterAbstractSelfTest.java |   131 +-
 ...airAffinityFunctionBackupFilterSelfTest.java |     9 +
 ...ousAffinityFunctionBackupFilterSelfTest.java |     9 +
 .../CacheJdbcPojoStoreAbstractSelfTest.java     |   161 +-
 ...heJdbcPojoStoreBinaryMarshallerSelfTest.java |    26 +-
 ...reBinaryMarshallerWithSqlEscapeSelfTest.java |    28 +
 ...dbcPojoStoreOptimizedMarshallerSelfTest.java |     2 +-
 ...ptimizedMarshallerWithSqlEscapeSelfTest.java |    28 +
 .../ignite/igfs/IgfsFragmentizerSelfTest.java   |     2 -
 .../apache/ignite/igfs/IgfsPathSelfTest.java    |     6 -
 .../ignite/internal/ClusterGroupSelfTest.java   |    32 +-
 .../internal/ClusterNodeMetricsSelfTest.java    |   101 +-
 .../internal/GridEventStorageSelfTest.java      |    97 +-
 .../ignite/internal/GridGetOrStartSelfTest.java |    70 +
 .../GridJobMasterLeaveAwareSelfTest.java        |     4 +-
 ...ectionLocalJobMultipleArgumentsSelfTest.java |     4 +-
 .../GridTaskFailoverAffinityRunTest.java        |     2 +-
 .../IgniteClientReconnectCacheTest.java         |    35 +
 ...eClientReconnectContinuousProcessorTest.java |    60 +-
 .../IgniteComputeEmptyClusterGroupTest.java     |     8 +-
 .../IgniteLocalNodeMapBeforeStartTest.java      |    82 +
 ...RoundRobinErrorAfterClientReconnectTest.java |    99 +
 .../MarshallerContextLockingSelfTest.java       |   139 +
 .../ignite/internal/binary/AffinityKey.java     |    69 +
 .../binary/BinaryMarshallerSelfTest.java        |   154 +-
 .../BinaryObjectBuilderAdditionalSelfTest.java  |    92 +-
 .../binary/BinaryObjectToStringSelfTest.java    |    92 +
 .../internal/binary/BinaryTreeSelfTest.java     |   341 +
 .../binary/GridBinaryAffinityKeySelfTest.java   |    19 +-
 ...aultBinaryMappersBinaryMetaDataSelfTest.java |    17 +
 ...GridManagerLocalMessageListenerSelfTest.java |   222 +
 .../managers/GridManagerStopSelfTest.java       |     2 +
 .../GridDiscoveryManagerAliveCacheSelfTest.java |     4 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |    63 +
 .../discovery/GridDiscoveryManagerSelfTest.java |     6 +-
 .../cache/CacheAffinityCallSelfTest.java        |    42 +-
 .../cache/CacheGetEntryAbstractTest.java        |    40 +-
 .../processors/cache/CachePutIfAbsentTest.java  |   161 +
 .../CacheSerializableTransactionsTest.java      |    75 +-
 .../CacheStartupInDeploymentModesTest.java      |   230 +
 .../CacheSwapUnswapGetTestSmallQueueSize.java   |    35 +
 .../processors/cache/CacheTxFastFinishTest.java |     2 +-
 .../CacheTxNotAllowReadFromBackupTest.java      |   297 +
 ...idAbstractCacheInterceptorRebalanceTest.java |   356 +
 .../cache/GridCacheAbstractFullApiSelfTest.java |   849 +-
 .../cache/GridCacheAbstractMetricsSelfTest.java |     2 +-
 .../cache/GridCacheAbstractSelfTest.java        |   243 +-
 ...acheAbstractUsersAffinityMapperSelfTest.java |     2 +-
 .../GridCacheInterceptorAbstractSelfTest.java   |     2 +-
 ...heInterceptorAtomicOffheapRebalanceTest.java |    30 +
 ...GridCacheInterceptorAtomicRebalanceTest.java |    36 +
 ...ceptorTransactionalOffheapRebalanceTest.java |    35 +
 ...heInterceptorTransactionalRebalanceTest.java |    36 +
 .../cache/GridCacheOffHeapCleanupTest.java      |   169 +
 ...ridCacheStoreManagerDeserializationTest.java |     1 +
 .../cache/GridCacheStoreValueBytesSelfTest.java |     2 +-
 .../cache/GridCacheSwapCleanupTest.java         |    99 +
 .../processors/cache/GridCacheTestEntryEx.java  |    11 +-
 .../GridCacheTtlManagerEvictionSelfTest.java    |   160 +
 .../GridCacheTtlManagerNotificationTest.java    |   297 +
 .../cache/GridCacheUtilsSelfTest.java           |    64 +-
 ...calCacheStoreManagerDeserializationTest.java |     2 +-
 .../cache/GridLocalIgniteSerializationTest.java |   378 +
 .../processors/cache/H2CacheStoreStrategy.java  |   468 +
 .../cache/IgniteCacheAbstractTest.java          |     2 +-
 .../cache/IgniteCacheAtomicPeekModesTest.java   |     2 +-
 .../IgniteCacheConfigVariationsFullApiTest.java |   286 +-
 .../cache/IgniteCacheCreateRestartSelfTest.java |     2 -
 ...niteCacheExpireAndUpdateConsistencyTest.java |     7 +
 .../IgniteCacheInterceptorSelfTestSuite.java    |     5 +
 ...gniteCacheInvokeReadThroughAbstractTest.java |   382 +
 ...iteCacheInvokeReadThroughSingleNodeTest.java |   106 +
 .../cache/IgniteCacheInvokeReadThroughTest.java |   182 +-
 .../cache/IgniteCachePeekModesAbstractTest.java |   471 +-
 .../IgniteCacheReadThroughEvictionSelfTest.java |     2 +-
 .../IgniteCacheReadThroughStoreCallTest.java    |   288 +
 ...eDynamicCacheStartNoExchangeTimeoutTest.java |     2 +
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   217 +-
 ...niteDynamicCacheStartStopConcurrentTest.java |     2 +-
 .../cache/IgniteTxConfigCacheSelfTest.java      |    91 +-
 .../IgniteTxExceptionAbstractSelfTest.java      |     1 +
 .../processors/cache/MapCacheStoreStrategy.java |   145 +
 .../MarshallerCacheJobRunNodeRestartTest.java   |     2 +-
 .../cache/TestCacheStoreStrategy.java           |    96 +
 .../cache/WithKeepBinaryCacheFullApiTest.java   |    10 +-
 .../CacheKeepBinaryWithInterceptorTest.java     |   419 +
 .../GridCacheBinaryObjectsAbstractSelfTest.java |   115 +-
 .../GridCacheQueueApiSelfAbstractTest.java      |    66 +
 .../IgniteCountDownLatchAbstractSelfTest.java   |   158 +-
 .../IgniteSemaphoreAbstractSelfTest.java        |    25 +-
 ...SemaphoreFailoverSafeReleasePermitsTest.java |   129 +
 .../CacheLateAffinityAssignmentTest.java        |     4 -
 .../GridCacheTransformEventSelfTest.java        |    66 +-
 .../IgniteCacheConnectionRecoveryTest.java      |   205 +
 .../IgniteCacheMessageRecoveryAbstractTest.java |    14 +-
 ...eCacheMessageRecoveryIdleConnectionTest.java |   157 +
 .../IgniteCacheMessageWriteTimeoutTest.java     |   129 +
 .../IgniteCacheTxIteratorSelfTest.java          |   241 +
 .../IgniteTxTimeoutAbstractTest.java            |     8 +-
 .../dht/IgniteCacheConcurrentPutGetRemove.java  |   201 +
 ...artitionedBackupNodeFailureRecoveryTest.java |   193 +
 .../IgniteCachePutRetryAbstractSelfTest.java    |    39 +-
 ...gniteCachePutRetryTransactionalSelfTest.java |    75 +-
 ...tionedMultiNodeLongTxTimeoutFullApiTest.java |    34 +
 ...nabledMultiNodeLongTxTimeoutFullApiTest.java |    41 +
 ...tomicClientOnlyMultiNodeFullApiSelfTest.java |    17 +-
 ...eAtomicNearOnlyMultiNodeFullApiSelfTest.java |    17 +-
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |   170 +-
 ...ePartitionedBasicStoreMultiNodeSelfTest.java |     2 +
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |     2 +-
 .../rebalancing/CacheNodeSafeAssertion.java     |   118 +
 .../GridCacheRebalancingOrderingTest.java       |   916 ++
 ...cheRebalancingPartitionDistributionTest.java |   149 +
 .../IgniteCacheSyncRebalanceModeSelfTest.java   |   116 +
 .../IgniteCacheExpiryPolicyAbstractTest.java    |    14 +-
 .../IgniteCacheExpiryPolicyTestSuite.java       |     5 +
 ...eCacheOnlyOneTtlCleanupThreadExistsTest.java |   102 +
 .../expiry/IgniteCacheTtlCleanupSelfTest.java   |     2 +-
 .../IgniteCacheLoaderWriterAbstractTest.java    |    10 +
 ...CacheLocalOffHeapAndSwapMetricsSelfTest.java |   415 -
 .../CacheOffHeapAndSwapMetricsSelfTest.java     |   621 +
 .../local/GridCacheLocalTxTimeoutSelfTest.java  |     5 +-
 .../GridCacheQueryTransformerSelfTest.java      |   575 +
 .../cache/query/IndexingSpiQuerySelfTest.java   |   218 +
 .../cache/query/IndexingSpiQueryTxSelfTest.java |   162 +
 ...eContinuousQueryAsyncFailoverTxSelfTest.java |     5 +
 ...ContinuousQueryFailoverAbstractSelfTest.java |   324 +-
 .../CacheContinuousQueryFailoverTxSelfTest.java |     5 +
 .../CacheContinuousQueryVariationsTest.java     |   949 ++
 ...eEntryProcessorExternalizableFailedTest.java |   588 +
 .../CacheEntryProcessorNonSerializableTest.java |   410 +
 ...CacheKeepBinaryIterationNearEnabledTest.java |    44 +
 ...acheKeepBinaryIterationStoreEnabledTest.java |    90 +
 ...CacheKeepBinaryIterationSwapEnabledTest.java |    56 +
 .../CacheKeepBinaryIterationTest.java           |   471 +
 ...yRemoteFilterMissingInClassPathSelfTest.java |   237 +
 ...ridCacheContinuousQueryAbstractSelfTest.java |    49 +-
 ...eContinuousQueryMultiNodesFilteringTest.java |   439 +
 ...dCacheContinuousQueryNodesFilteringTest.java |   168 +
 ...niteCacheContinuousQueryBackupQueueTest.java |   299 +
 ...eCacheContinuousQueryImmutableEntryTest.java |   205 +
 ...teCacheContinuousQueryNoUnsubscribeTest.java |   153 +
 .../transactions/DepthFirstSearchTest.java      |   100 +-
 .../TxDeadlockDetectionNoHangsTest.java         |   246 +
 .../transactions/TxDeadlockDetectionTest.java   |    13 +-
 ...timisticDeadlockDetectionCrossCacheTest.java |   257 +
 .../TxOptimisticDeadlockDetectionTest.java      |   574 +
 ...simisticDeadlockDetectionCrossCacheTest.java |   165 +-
 .../TxPessimisticDeadlockDetectionTest.java     |    50 +-
 .../cluster/GridAddressResolverSelfTest.java    |    97 +
 .../IgniteNoCustomEventsOnNodeStart.java        |    85 +
 .../datastreamer/DataStreamerTimeoutTest.java   |   163 +
 ...faultIgfsSecondaryFileSystemTestAdapter.java |   117 +
 .../igfs/IgfsAbstractBaseSelfTest.java          |  1093 ++
 .../processors/igfs/IgfsAbstractSelfTest.java   |  1782 +--
 .../IgfsAtomicPrimaryMultiNodeSelfTest.java     |    39 +
 .../IgfsAtomicPrimaryOffheapTieredSelfTest.java |    39 +
 .../IgfsAtomicPrimaryOffheapValuesSelfTest.java |    39 +
 .../igfs/IgfsAtomicPrimarySelfTest.java         |    39 +
 .../igfs/IgfsBackupFailoverSelfTest.java        |     2 +-
 ...sCachePerBlockLruEvictionPolicySelfTest.java |     7 +-
 .../igfs/IgfsClientCacheSelfTest.java           |   139 -
 .../igfs/IgfsDataManagerSelfTest.java           |    18 +-
 .../igfs/IgfsDualAbstractSelfTest.java          |   359 +-
 .../igfs/IgfsDualAsyncClientSelfTest.java       |    28 +
 .../igfs/IgfsDualSyncClientSelfTest.java        |    28 +
 .../igfs/IgfsExUniversalFileSystemAdapter.java  |   101 -
 .../processors/igfs/IgfsIgniteMock.java         |   520 +
 ...SecondaryFileSystemDualAbstractSelfTest.java |   297 +
 ...ondaryFileSystemDualAsyncClientSelfTest.java |    28 +
 ...calSecondaryFileSystemDualAsyncSelfTest.java |    32 +
 ...condaryFileSystemDualSyncClientSelfTest.java |    28 +
 ...ocalSecondaryFileSystemDualSyncSelfTest.java |    32 +
 ...lSecondaryFileSystemProxyClientSelfTest.java |    28 +
 ...fsLocalSecondaryFileSystemProxySelfTest.java |   267 +
 ...IgfsLocalSecondaryFileSystemTestAdapter.java |   164 +
 .../processors/igfs/IgfsMaxSizeSelfTest.java    |   122 +
 .../igfs/IgfsMetaManagerSelfTest.java           |    28 +-
 .../processors/igfs/IgfsMetricsSelfTest.java    |    75 +-
 .../internal/processors/igfs/IgfsMock.java      |   405 +
 .../igfs/IgfsModeResolverSelfTest.java          |   161 +-
 .../processors/igfs/IgfsModesSelfTest.java      |    64 +-
 .../processors/igfs/IgfsOneClientNodeTest.java  |     7 +-
 .../igfs/IgfsPrimaryClientSelfTest.java         |    30 +
 ...PrimaryRelaxedConsistencyClientSelfTest.java |    28 +
 .../processors/igfs/IgfsProcessorSelfTest.java  |    11 +-
 .../igfs/IgfsProcessorValidationSelfTest.java   |    64 +-
 .../processors/igfs/IgfsProxySelfTest.java      |    32 +
 .../IgfsSecondaryFileSystemTestAdapter.java     |   118 +
 .../processors/igfs/IgfsSizeSelfTest.java       |   133 -
 .../processors/igfs/IgfsStreamsSelfTest.java    |     9 +-
 .../processors/igfs/IgfsTaskSelfTest.java       |     2 +-
 .../igfs/UniversalFileSystemAdapter.java        |    98 -
 .../igfs/benchmark/IgfsBenchmark.java           |   561 +
 .../odbc/OdbcEscapeSequenceSelfTest.java        |   778 +
 .../odbc/OdbcProcessorValidationSelfTest.java   |    37 +-
 .../GridCacheAtomicCommandHandlerSelfTest.java  |    39 +
 .../cache/GridCacheCommandHandlerSelfTest.java  |    20 +-
 .../handlers/log/GridLogCommandHandlerTest.java |     5 +-
 .../query/GridQueryCommandHandlerTest.java      |   191 +
 .../service/GridServiceClientNodeTest.java      |   102 +-
 ...rviceDeploymentExceptionPropagationTest.java |    80 +
 ...yment2ClassLoadersDefaultMarshallerTest.java |   259 +
 ...eployment2ClassLoadersJdkMarshallerTest.java |    31 +
 ...ent2ClassLoadersOptimizedMarshallerTest.java |    31 +
 ...oymentClassLoadingDefaultMarshallerTest.java |   212 +
 ...DeploymentClassLoadingJdkMarshallerTest.java |    31 +
 ...mentClassLoadingOptimizedMarshallerTest.java |    31 +
 .../service/IgniteServiceReassignmentTest.java  |   250 +
 .../ServicePredicateAccessCacheTest.java        |     4 +-
 ...artupWithSpecifiedWorkDirectorySelfTest.java |   166 -
 .../internal/util/IgniteUtilsSelfTest.java      |    15 +
 .../util/future/GridFutureAdapterSelfTest.java  |     3 +
 .../ipc/shmem/IgfsSharedMemoryTestServer.java   |     4 +-
 .../IpcSharedMemoryCrashDetectionSelfTest.java  |     9 +-
 .../IpcSharedMemoryBenchmarkReader.java         |     2 +-
 .../unsafe/GridOffheapSnapTreeSelfTest.java     |     2 +-
 .../loadtests/hashmap/GridCacheTestContext.java |     2 +
 .../ignite/logger/java/JavaLoggerTest.java      |     3 +-
 .../marshaller/MarshallerContextSelfTest.java   |    90 +
 .../marshaller/MarshallerContextTestImpl.java   |    28 +-
 .../OptimizedMarshallerNodeFailoverTest.java    |     4 -
 .../ignite/platform/PlatformStringTestTask.java |    67 +
 .../spi/GridTcpSpiForwardingSelfTest.java       |   237 +-
 ...heckpointSpiMultipleDirectoriesSelfTest.java |     6 +-
 .../GridAbstractCommunicationSelfTest.java      |     2 -
 ...mmunicationSpiConcurrentConnectSelfTest.java |    14 +
 ...nicationSpiConcurrentConnectSslSelfTest.java |    35 +
 ...cpCommunicationSpiMultithreadedSelfTest.java |     2 -
 ...GridTcpCommunicationSpiRecoverySelfTest.java |    14 +
 ...dTcpCommunicationSpiRecoverySslSelfTest.java |    35 +
 ...CommunicationSpiSslSmallBuffersSelfTest.java |    43 +
 .../tcp/IgniteCacheSslStartStopSelfTest.java    |     1 -
 .../tcp/TcpClientDiscoverySpiMulticastTest.java |     2 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |   222 +
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |    67 +-
 .../tcp/TcpDiscoverySnapshotHistoryTest.java    |    14 +-
 .../jdbc/TcpDiscoveryJdbcIpFinderSelfTest.java  |     2 +-
 .../TcpDiscoveryMulticastIpFinderSelfTest.java  |     6 +-
 .../TcpDiscoverySharedFsIpFinderSelfTest.java   |    25 +
 .../vm/TcpDiscoveryVmIpFinderSelfTest.java      |    75 +
 .../spi/failover/GridFailoverTestContext.java   |     6 +
 .../GridInternalTasksLoadBalancingSelfTest.java |   201 +
 .../GridSwapSpaceSpiAbstractSelfTest.java       |     2 -
 .../file/GridFileSwapSpaceSpiSelfTest.java      |    89 +
 .../testframework/GridSpiTestContext.java       |   115 +-
 .../ignite/testframework/GridTestUtils.java     |    39 +-
 .../ignite/testframework/IgniteTestSuite.java   |   429 +
 .../assertions/AlwaysAssertion.java             |    29 +
 .../testframework/assertions/Assertion.java     |    31 +
 .../testframework/assertions/package-info.java  |    22 +
 .../configvariations/ConfigVariations.java      |     6 +-
 .../testframework/junits/GridAbstractTest.java  |    65 +-
 .../junits/GridTestKernalContext.java           |     2 +
 ...IgniteCacheConfigVariationsAbstractTest.java |    67 +-
 .../ignite/testframework/junits/IgniteMock.java |    25 +
 .../junits/IgniteTestResources.java             |     2 -
 .../junits/common/GridCommonAbstractTest.java   |    59 +-
 .../common/GridRollingRestartAbstractTest.java  |   324 +
 .../multijvm/IgniteCacheProcessProxy.java       |   160 +-
 .../multijvm/IgniteClusterProcessProxy.java     |     5 +
 .../junits/multijvm/IgniteProcessProxy.java     |    39 +-
 .../junits/spi/GridSpiAbstractTest.java         |     2 -
 .../ignite/testsuites/IgniteBasicTestSuite.java |    12 +-
 .../testsuites/IgniteBinaryBasicTestSuite.java  |     2 -
 .../testsuites/IgniteBinaryCacheTestSuite.java  |     2 +
 .../IgniteBinaryObjectsTestSuite.java           |     4 +
 .../IgniteCacheDataStructuresSelfTestSuite.java |     2 +
 .../IgniteCacheFailoverTestSuite.java           |     2 +
 .../IgniteCacheFullApiSelfTestSuite.java        |     4 +
 .../IgniteCacheMetricsSelfTestSuite.java        |     4 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |    20 +
 .../testsuites/IgniteCacheTestSuite2.java       |     5 +
 .../testsuites/IgniteCacheTestSuite3.java       |     2 +
 .../testsuites/IgniteCacheTestSuite4.java       |    10 +
 .../testsuites/IgniteCacheTestSuite5.java       |     7 +
 .../testsuites/IgniteComputeGridTestSuite.java  |     2 +
 ...iteContinuousQueryConfigVariationsSuite.java |    60 +
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |    39 +-
 .../apache/ignite/testsuites/IgniteIgnore.java  |    40 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    21 +-
 .../testsuites/IgniteRestHandlerTestSuite.java  |     4 +
 .../IgniteSpiCommunicationSelfTestSuite.java    |    11 +-
 .../IgniteSpiLoadBalancingSelfTestSuite.java    |    34 +-
 .../ignite/testsuites/IgniteSpiTestSuite.java   |     6 +-
 .../testsuites/IgniteUtilSelfTestSuite.java     |     2 +
 .../TxDeadlockDetectionTestSuite.java           |     6 +
 .../ignite/thread/IgniteThreadPoolSizeTest.java |   131 +
 .../apache/ignite/util/GridLogThrottleTest.java |    27 +-
 modules/docker/1.6.0/Dockerfile                 |    44 +
 modules/docker/1.6.0/run.sh                     |    51 +
 modules/docker/1.7.0/Dockerfile                 |    44 +
 modules/docker/1.7.0/run.sh                     |    51 +
 modules/docker/Dockerfile                       |    24 +-
 modules/extdata/p2p/pom.xml                     |     2 +-
 .../apache/ignite/tests/p2p/NoopService.java    |    41 +
 .../apache/ignite/tests/p2p/NoopService2.java   |    41 +
 .../CacheNoValueClassOnServerTestClient.java    |     2 -
 .../CacheConfigurationP2PTestClient.java        |     2 -
 .../extdata/uri/modules/uri-dependency/pom.xml  |     2 +-
 modules/extdata/uri/pom.xml                     |     2 +-
 modules/flink/pom.xml                           |     2 +-
 .../apache/ignite/sink/flink/IgniteSink.java    |     2 +-
 modules/flume/README.txt                        |     2 +-
 modules/flume/pom.xml                           |     2 +-
 modules/gce/pom.xml                             |     2 +-
 modules/geospatial/pom.xml                      |     2 +-
 .../query/h2/opt/GridH2SpatialIndex.java        |    74 +-
 modules/hadoop/pom.xml                          |    51 +-
 .../hadoop/fs/BasicHadoopFileSystemFactory.java |   152 +-
 .../fs/CachingHadoopFileSystemFactory.java      |    55 +-
 .../hadoop/fs/HadoopFileSystemFactory.java      |    11 +-
 .../fs/IgniteHadoopFileSystemCounterWriter.java |    79 +-
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |   398 +-
 .../fs/KerberosHadoopFileSystemFactory.java     |    77 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |    54 +-
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |    76 +-
 .../IgniteHadoopClientProtocolProvider.java     |    47 +-
 .../mapreduce/IgniteHadoopMapReducePlanner.java |    66 +-
 .../IgniteHadoopWeightedMapReducePlanner.java   |   846 ++
 .../ignite/hadoop/util/BasicUserNameMapper.java |   112 +
 .../hadoop/util/ChainedUserNameMapper.java      |    94 +
 .../hadoop/util/KerberosUserNameMapper.java     |   137 +
 .../ignite/hadoop/util/UserNameMapper.java      |    35 +
 .../apache/ignite/hadoop/util/package-info.java |    22 +
 .../processors/hadoop/HadoopAttributes.java     |   168 +
 .../processors/hadoop/HadoopClassLoader.java    |   992 --
 .../processors/hadoop/HadoopCommonUtils.java    |   154 +
 .../processors/hadoop/HadoopContext.java        |     1 -
 .../processors/hadoop/HadoopDefaultJobInfo.java |   157 -
 .../processors/hadoop/HadoopExternalSplit.java  |    96 +
 .../processors/hadoop/HadoopHelperImpl.java     |   133 +
 .../hadoop/HadoopMapReduceCounterGroup.java     |   123 -
 .../hadoop/HadoopMapReduceCounters.java         |   228 -
 .../processors/hadoop/HadoopProcessor.java      |   150 +-
 .../processors/hadoop/HadoopSplitWrapper.java   |   128 +
 .../internal/processors/hadoop/HadoopUtils.java |   355 -
 .../hadoop/counter/HadoopCounterAdapter.java    |     1 +
 .../counter/HadoopPerformanceCounter.java       |    12 +-
 .../hadoop/delegate/HadoopDelegateUtils.java    |   138 +
 .../HadoopFileSystemCounterWriterDelegate.java  |    36 +
 .../HadoopFileSystemFactoryDelegate.java        |    36 +
 .../HadoopIgfsSecondaryFileSystemDelegate.java  |    28 +
 .../hadoop/fs/HadoopFileSystemCacheUtils.java   |   242 -
 .../hadoop/fs/HadoopFileSystemsUtils.java       |    51 -
 .../hadoop/fs/HadoopLazyConcurrentMap.java      |   212 -
 .../hadoop/fs/HadoopLocalFileSystemV1.java      |    39 -
 .../hadoop/fs/HadoopLocalFileSystemV2.java      |    88 -
 .../processors/hadoop/fs/HadoopParameters.java  |    94 -
 .../hadoop/fs/HadoopRawLocalFileSystem.java     |   314 -
 .../processors/hadoop/igfs/HadoopIgfs.java      |   202 -
 .../igfs/HadoopIgfsCommunicationException.java  |    57 -
 .../processors/hadoop/igfs/HadoopIgfsEx.java    |    93 -
 .../hadoop/igfs/HadoopIgfsFuture.java           |    97 -
 .../hadoop/igfs/HadoopIgfsInProc.java           |   510 -
 .../hadoop/igfs/HadoopIgfsInputStream.java      |   629 -
 .../processors/hadoop/igfs/HadoopIgfsIo.java    |    76 -
 .../processors/hadoop/igfs/HadoopIgfsIpcIo.java |   624 -
 .../hadoop/igfs/HadoopIgfsIpcIoListener.java    |    36 -
 .../hadoop/igfs/HadoopIgfsJclLogger.java        |   116 -
 .../hadoop/igfs/HadoopIgfsOutProc.java          |   524 -
 .../hadoop/igfs/HadoopIgfsOutputStream.java     |   201 -
 .../hadoop/igfs/HadoopIgfsProperties.java       |    86 -
 .../hadoop/igfs/HadoopIgfsProxyInputStream.java |   337 -
 .../igfs/HadoopIgfsProxyOutputStream.java       |   165 -
 ...fsSecondaryFileSystemPositionedReadable.java |   105 -
 .../hadoop/igfs/HadoopIgfsStreamDelegate.java   |    96 -
 .../igfs/HadoopIgfsStreamEventListener.java     |    39 -
 .../processors/hadoop/igfs/HadoopIgfsUtils.java |   174 -
 .../hadoop/igfs/HadoopIgfsWrapper.java          |   552 -
 .../impl/HadoopMapReduceCounterGroup.java       |   124 +
 .../hadoop/impl/HadoopMapReduceCounters.java    |   229 +
 .../processors/hadoop/impl/HadoopUtils.java     |   331 +
 .../HadoopBasicFileSystemFactoryDelegate.java   |   178 +
 .../HadoopCachingFileSystemFactoryDelegate.java |    75 +
 .../HadoopDefaultFileSystemFactoryDelegate.java |    62 +
 ...doopFileSystemCounterWriterDelegateImpl.java |   108 +
 ...doopIgfsSecondaryFileSystemDelegateImpl.java |   471 +
 ...HadoopKerberosFileSystemFactoryDelegate.java |   117 +
 .../impl/fs/HadoopFileSystemCacheUtils.java     |   243 +
 .../hadoop/impl/fs/HadoopFileSystemsUtils.java  |    51 +
 .../hadoop/impl/fs/HadoopLazyConcurrentMap.java |   210 +
 .../hadoop/impl/fs/HadoopLocalFileSystemV1.java |    40 +
 .../hadoop/impl/fs/HadoopLocalFileSystemV2.java |    89 +
 .../hadoop/impl/fs/HadoopParameters.java        |    94 +
 .../impl/fs/HadoopRawLocalFileSystem.java       |   315 +
 .../processors/hadoop/impl/igfs/HadoopIgfs.java |   203 +
 .../igfs/HadoopIgfsCommunicationException.java  |    57 +
 .../hadoop/impl/igfs/HadoopIgfsEx.java          |    94 +
 .../hadoop/impl/igfs/HadoopIgfsFuture.java      |    97 +
 .../hadoop/impl/igfs/HadoopIgfsInProc.java      |   511 +
 .../hadoop/impl/igfs/HadoopIgfsInputStream.java |   630 +
 .../hadoop/impl/igfs/HadoopIgfsIo.java          |    76 +
 .../hadoop/impl/igfs/HadoopIgfsIpcIo.java       |   625 +
 .../impl/igfs/HadoopIgfsIpcIoListener.java      |    36 +
 .../hadoop/impl/igfs/HadoopIgfsJclLogger.java   |   116 +
 .../hadoop/impl/igfs/HadoopIgfsOutProc.java     |   525 +
 .../impl/igfs/HadoopIgfsOutputStream.java       |   202 +
 .../hadoop/impl/igfs/HadoopIgfsProperties.java  |    90 +
 .../impl/igfs/HadoopIgfsProxyInputStream.java   |   338 +
 .../impl/igfs/HadoopIgfsProxyOutputStream.java  |   166 +
 ...fsSecondaryFileSystemPositionedReadable.java |   106 +
 .../impl/igfs/HadoopIgfsStreamDelegate.java     |    96 +
 .../igfs/HadoopIgfsStreamEventListener.java     |    39 +
 .../hadoop/impl/igfs/HadoopIgfsUtils.java       |   175 +
 .../hadoop/impl/igfs/HadoopIgfsWrapper.java     |   554 +
 .../hadoop/impl/proto/HadoopClientProtocol.java |   354 +
 .../hadoop/impl/v1/HadoopV1CleanupTask.java     |    65 +
 .../hadoop/impl/v1/HadoopV1Counter.java         |   107 +
 .../hadoop/impl/v1/HadoopV1MapTask.java         |   122 +
 .../hadoop/impl/v1/HadoopV1OutputCollector.java |   138 +
 .../hadoop/impl/v1/HadoopV1Partitioner.java     |    44 +
 .../hadoop/impl/v1/HadoopV1ReduceTask.java      |   101 +
 .../hadoop/impl/v1/HadoopV1Reporter.java        |    81 +
 .../hadoop/impl/v1/HadoopV1SetupTask.java       |    57 +
 .../hadoop/impl/v1/HadoopV1Splitter.java        |   103 +
 .../processors/hadoop/impl/v1/HadoopV1Task.java |    98 +
 .../processors/hadoop/impl/v2/HadoopDaemon.java |   126 +
 .../impl/v2/HadoopSerializationWrapper.java     |   139 +
 .../impl/v2/HadoopShutdownHookManager.java      |    98 +
 .../hadoop/impl/v2/HadoopV2CleanupTask.java     |    73 +
 .../hadoop/impl/v2/HadoopV2Context.java         |   244 +
 .../hadoop/impl/v2/HadoopV2Counter.java         |    89 +
 .../processors/hadoop/impl/v2/HadoopV2Job.java  |   457 +
 .../impl/v2/HadoopV2JobResourceManager.java     |   324 +
 .../hadoop/impl/v2/HadoopV2MapTask.java         |    99 +
 .../hadoop/impl/v2/HadoopV2Partitioner.java     |    44 +
 .../hadoop/impl/v2/HadoopV2ReduceTask.java      |    91 +
 .../hadoop/impl/v2/HadoopV2SetupTask.java       |    66 +
 .../hadoop/impl/v2/HadoopV2Splitter.java        |   112 +
 .../processors/hadoop/impl/v2/HadoopV2Task.java |   186 +
 .../hadoop/impl/v2/HadoopV2TaskContext.java     |   563 +
 .../impl/v2/HadoopWritableSerialization.java    |    76 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |    84 +-
 .../planner/HadoopAbstractMapReducePlanner.java |   116 +
 .../planner/HadoopDefaultMapReducePlan.java     |     7 +-
 .../planner/HadoopMapReducePlanGroup.java       |   150 +
 .../planner/HadoopMapReducePlanTopology.java    |    89 +
 .../hadoop/proto/HadoopClientProtocol.java      |   349 -
 .../shuffle/collections/HadoopMultimapBase.java |    90 +-
 .../external/HadoopExternalTaskExecutor.java    |    13 +-
 .../child/HadoopChildProcessRunner.java         |    16 +-
 .../child/HadoopExternalProcessStarter.java     |    10 +-
 .../HadoopExternalCommunication.java            |    16 +-
 .../communication/HadoopMarshallerFilter.java   |    13 +-
 .../hadoop/v1/HadoopV1CleanupTask.java          |    64 -
 .../processors/hadoop/v1/HadoopV1Counter.java   |   106 -
 .../processors/hadoop/v1/HadoopV1MapTask.java   |   122 -
 .../hadoop/v1/HadoopV1OutputCollector.java      |   137 -
 .../hadoop/v1/HadoopV1Partitioner.java          |    44 -
 .../hadoop/v1/HadoopV1ReduceTask.java           |   101 -
 .../processors/hadoop/v1/HadoopV1Reporter.java  |    81 -
 .../processors/hadoop/v1/HadoopV1SetupTask.java |    56 -
 .../processors/hadoop/v1/HadoopV1Splitter.java  |   102 -
 .../processors/hadoop/v1/HadoopV1Task.java      |    97 -
 .../processors/hadoop/v2/HadoopDaemon.java      |   126 -
 .../hadoop/v2/HadoopExternalSplit.java          |    89 -
 .../hadoop/v2/HadoopSerializationWrapper.java   |   138 -
 .../hadoop/v2/HadoopShutdownHookManager.java    |    98 -
 .../hadoop/v2/HadoopSplitWrapper.java           |   119 -
 .../hadoop/v2/HadoopV2CleanupTask.java          |    72 -
 .../processors/hadoop/v2/HadoopV2Context.java   |   243 -
 .../processors/hadoop/v2/HadoopV2Counter.java   |    88 -
 .../processors/hadoop/v2/HadoopV2Job.java       |   435 -
 .../hadoop/v2/HadoopV2JobResourceManager.java   |   322 -
 .../processors/hadoop/v2/HadoopV2MapTask.java   |    99 -
 .../hadoop/v2/HadoopV2Partitioner.java          |    44 -
 .../hadoop/v2/HadoopV2ReduceTask.java           |    91 -
 .../processors/hadoop/v2/HadoopV2SetupTask.java |    65 -
 .../processors/hadoop/v2/HadoopV2Splitter.java  |   111 -
 .../processors/hadoop/v2/HadoopV2Task.java      |   185 -
 .../hadoop/v2/HadoopV2TaskContext.java          |   559 -
 .../hadoop/v2/HadoopWritableSerialization.java  |    75 -
 .../HadoopClientProtocolEmbeddedSelfTest.java   |    35 -
 .../hadoop/HadoopClientProtocolSelfTest.java    |   654 -
 .../hadoop/cache/HadoopTxConfigCacheTest.java   |    42 -
 ...KerberosHadoopFileSystemFactorySelfTest.java |   121 -
 .../ignite/igfs/Hadoop1DualAbstractTest.java    |   113 -
 .../igfs/Hadoop1OverIgfsDualAsyncTest.java      |    30 -
 .../igfs/Hadoop1OverIgfsDualSyncTest.java       |    30 -
 .../igfs/HadoopFIleSystemFactorySelfTest.java   |   326 -
 ...oopFileSystemUniversalFileSystemAdapter.java |   131 -
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |  2040 ---
 ...Igfs20FileSystemLoopbackPrimarySelfTest.java |    74 -
 ...oopIgfs20FileSystemShmemPrimarySelfTest.java |    74 -
 .../igfs/HadoopIgfsDualAbstractSelfTest.java    |   321 -
 .../igfs/HadoopIgfsDualAsyncSelfTest.java       |    32 -
 .../ignite/igfs/HadoopIgfsDualSyncSelfTest.java |    32 -
 ...oopSecondaryFileSystemConfigurationTest.java |   575 -
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |   285 -
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |   223 -
 .../IgniteHadoopFileSystemAbstractSelfTest.java |  2433 ---
 .../IgniteHadoopFileSystemClientSelfTest.java   |   212 -
 ...IgniteHadoopFileSystemHandshakeSelfTest.java |   389 -
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |   214 -
 .../IgniteHadoopFileSystemLoggerSelfTest.java   |   298 -
 ...niteHadoopFileSystemLoggerStateSelfTest.java |   329 -
 ...adoopFileSystemLoopbackAbstractSelfTest.java |    46 -
 ...SystemLoopbackEmbeddedDualAsyncSelfTest.java |    33 -
 ...eSystemLoopbackEmbeddedDualSyncSelfTest.java |    33 -
 ...leSystemLoopbackEmbeddedPrimarySelfTest.java |    33 -
 ...SystemLoopbackEmbeddedSecondarySelfTest.java |    34 -
 ...SystemLoopbackExternalDualAsyncSelfTest.java |    33 -
 ...eSystemLoopbackExternalDualSyncSelfTest.java |    33 -
 ...leSystemLoopbackExternalPrimarySelfTest.java |    33 -
 ...SystemLoopbackExternalSecondarySelfTest.java |    34 -
 ...condaryFileSystemInitializationSelfTest.java |   214 -
 ...teHadoopFileSystemSecondaryModeSelfTest.java |   327 -
 ...teHadoopFileSystemShmemAbstractSelfTest.java |    91 -
 ...ileSystemShmemEmbeddedDualAsyncSelfTest.java |    33 -
 ...FileSystemShmemEmbeddedDualSyncSelfTest.java |    33 -
 ...pFileSystemShmemEmbeddedPrimarySelfTest.java |    33 -
 ...ileSystemShmemEmbeddedSecondarySelfTest.java |    33 -
 ...ileSystemShmemExternalDualAsyncSelfTest.java |    33 -
 ...FileSystemShmemExternalDualSyncSelfTest.java |    33 -
 ...pFileSystemShmemExternalPrimarySelfTest.java |    33 -
 ...ileSystemShmemExternalSecondarySelfTest.java |    33 -
 .../hadoop/HadoopAbstractMapReduceTest.java     |   405 -
 .../hadoop/HadoopAbstractSelfTest.java          |   239 -
 .../hadoop/HadoopAbstractWordCountTest.java     |   175 -
 .../hadoop/HadoopClassLoaderTest.java           |   110 -
 .../hadoop/HadoopCommandLineTest.java           |   474 -
 .../HadoopDefaultMapReducePlannerSelfTest.java  |  1028 --
 .../processors/hadoop/HadoopErrorSimulator.java |   326 -
 .../hadoop/HadoopFileSystemsTest.java           |   155 -
 .../processors/hadoop/HadoopGroupingTest.java   |   307 -
 .../hadoop/HadoopJobTrackerSelfTest.java        |   345 -
 .../hadoop/HadoopMapReduceEmbeddedSelfTest.java |   253 -
 .../HadoopMapReduceErrorResilienceTest.java     |   154 -
 .../processors/hadoop/HadoopMapReduceTest.java  |    66 -
 .../hadoop/HadoopNoHadoopMapReduceTest.java     |    47 -
 .../hadoop/HadoopPopularWordsTest.java          |   298 -
 .../HadoopSerializationWrapperSelfTest.java     |    79 -
 .../processors/hadoop/HadoopSharedMap.java      |     1 +
 .../hadoop/HadoopSnappyFullMapReduceTest.java   |    36 -
 .../processors/hadoop/HadoopSnappyTest.java     |   102 -
 .../hadoop/HadoopSortingExternalTest.java       |    46 -
 .../processors/hadoop/HadoopSortingTest.java    |   303 -
 .../hadoop/HadoopSplitWrapperSelfTest.java      |    72 -
 .../processors/hadoop/HadoopStartup.java        |    54 -
 .../hadoop/HadoopTaskExecutionSelfTest.java     |   567 -
 .../hadoop/HadoopTasksAllVersionsTest.java      |   260 -
 .../processors/hadoop/HadoopTasksV1Test.java    |    58 -
 .../processors/hadoop/HadoopTasksV2Test.java    |    77 -
 .../hadoop/HadoopTestClassLoader.java           |   106 +
 .../hadoop/HadoopTestRoundRobinMrPlanner.java   |    71 -
 .../hadoop/HadoopTestTaskContext.java           |   228 -
 .../processors/hadoop/HadoopTestUtils.java      |   107 -
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   100 -
 .../hadoop/HadoopValidationSelfTest.java        |    53 -
 .../hadoop/books/alice-in-wonderland.txt        |  3735 -----
 .../processors/hadoop/books/art-of-war.txt      |  6982 ---------
 .../hadoop/books/huckleberry-finn.txt           | 11733 ---------------
 .../processors/hadoop/books/sherlock-holmes.txt | 13052 -----------------
 .../processors/hadoop/books/tom-sawyer.txt      |  8858 -----------
 .../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/HadoopWordCount1.java       |    94 -
 .../hadoop/examples/HadoopWordCount1Map.java    |    79 -
 .../hadoop/examples/HadoopWordCount1Reduce.java |    61 -
 .../hadoop/examples/HadoopWordCount2.java       |   111 -
 .../examples/HadoopWordCount2Combiner.java      |    45 -
 .../hadoop/examples/HadoopWordCount2Mapper.java |    88 -
 .../examples/HadoopWordCount2Reducer.java       |   113 -
 .../impl/HadoopAbstractMapReduceTest.java       |   430 +
 .../hadoop/impl/HadoopAbstractSelfTest.java     |   239 +
 .../impl/HadoopAbstractWordCountTest.java       |   175 +
 .../hadoop/impl/HadoopCommandLineTest.java      |   476 +
 .../HadoopDefaultMapReducePlannerSelfTest.java  |   619 +
 .../hadoop/impl/HadoopErrorSimulator.java       |   326 +
 .../hadoop/impl/HadoopFileSystemsTest.java      |   155 +
 .../hadoop/impl/HadoopGroupingTest.java         |   302 +
 .../hadoop/impl/HadoopJobTrackerSelfTest.java   |   334 +
 .../impl/HadoopMapReduceEmbeddedSelfTest.java   |   249 +
 .../HadoopMapReduceErrorResilienceTest.java     |   154 +
 .../hadoop/impl/HadoopMapReduceTest.java        |    66 +
 .../impl/HadoopNoHadoopMapReduceTest.java       |    47 +
 .../hadoop/impl/HadoopPlannerMockJob.java       |   182 +
 .../hadoop/impl/HadoopPopularWordsTest.java     |   298 +
 .../HadoopSerializationWrapperSelfTest.java     |    80 +
 .../impl/HadoopSnappyFullMapReduceTest.java     |    36 +
 .../hadoop/impl/HadoopSnappyTest.java           |   104 +
 .../hadoop/impl/HadoopSortingExternalTest.java  |    46 +
 .../hadoop/impl/HadoopSortingTest.java          |   304 +
 .../hadoop/impl/HadoopSplitWrapperSelfTest.java |    72 +
 .../processors/hadoop/impl/HadoopStartup.java   |    54 +
 .../impl/HadoopTaskExecutionSelfTest.java       |   550 +
 .../hadoop/impl/HadoopTasksAllVersionsTest.java |   264 +
 .../hadoop/impl/HadoopTasksV1Test.java          |    62 +
 .../hadoop/impl/HadoopTasksV2Test.java          |    81 +
 .../impl/HadoopTestRoundRobinMrPlanner.java     |    75 +
 .../hadoop/impl/HadoopTestTaskContext.java      |   233 +
 .../processors/hadoop/impl/HadoopTestUtils.java |   178 +
 .../hadoop/impl/HadoopTxConfigCacheTest.java    |    42 +
 .../hadoop/impl/HadoopUserLibsSelfTest.java     |   261 +
 .../hadoop/impl/HadoopV2JobSelfTest.java        |   108 +
 .../hadoop/impl/HadoopValidationSelfTest.java   |    53 +
 .../HadoopWeightedMapReducePlannerTest.java     |   602 +
 .../HadoopWeightedPlannerMapReduceTest.java     |    38 +
 .../hadoop/impl/books/alice-in-wonderland.txt   |  3735 +++++
 .../processors/hadoop/impl/books/art-of-war.txt |  6982 +++++++++
 .../hadoop/impl/books/huckleberry-finn.txt      | 11733 +++++++++++++++
 .../hadoop/impl/books/sherlock-holmes.txt       | 13052 +++++++++++++++++
 .../processors/hadoop/impl/books/tom-sawyer.txt |  8858 +++++++++++
 .../HadoopClientProtocolEmbeddedSelfTest.java   |    35 +
 ...opClientProtocolMultipleServersSelfTest.java |   324 +
 .../client/HadoopClientProtocolSelfTest.java    |   654 +
 .../hadoop/impl/examples/HadoopWordCount1.java  |    94 +
 .../impl/examples/HadoopWordCount1Map.java      |    79 +
 .../impl/examples/HadoopWordCount1Reduce.java   |    61 +
 .../hadoop/impl/examples/HadoopWordCount2.java  |   111 +
 .../impl/examples/HadoopWordCount2Combiner.java |    45 +
 .../impl/examples/HadoopWordCount2Mapper.java   |    88 +
 .../impl/examples/HadoopWordCount2Reducer.java  |   113 +
 ...KerberosHadoopFileSystemFactorySelfTest.java |   126 +
 .../impl/igfs/Hadoop1DualAbstractTest.java      |   163 +
 .../impl/igfs/Hadoop1OverIgfsDualAsyncTest.java |    32 +
 .../impl/igfs/Hadoop1OverIgfsDualSyncTest.java  |    32 +
 .../igfs/HadoopFIleSystemFactorySelfTest.java   |   345 +
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |  2047 +++
 ...Igfs20FileSystemLoopbackPrimarySelfTest.java |    77 +
 ...oopIgfs20FileSystemShmemPrimarySelfTest.java |    77 +
 .../igfs/HadoopIgfsDualAbstractSelfTest.java    |   328 +
 .../impl/igfs/HadoopIgfsDualAsyncSelfTest.java  |    32 +
 .../impl/igfs/HadoopIgfsDualSyncSelfTest.java   |    32 +
 ...adoopIgfsSecondaryFileSystemTestAdapter.java |   153 +
 ...oopSecondaryFileSystemConfigurationTest.java |   595 +
 .../hadoop/impl/igfs/IgfsEventsTestSuite.java   |   289 +
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |   226 +
 .../IgniteHadoopFileSystemAbstractSelfTest.java |  2435 +++
 .../IgniteHadoopFileSystemClientSelfTest.java   |   216 +
 ...IgniteHadoopFileSystemHandshakeSelfTest.java |   393 +
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |   215 +
 .../IgniteHadoopFileSystemLoggerSelfTest.java   |   299 +
 ...niteHadoopFileSystemLoggerStateSelfTest.java |   332 +
 ...adoopFileSystemLoopbackAbstractSelfTest.java |    50 +
 ...SystemLoopbackEmbeddedDualAsyncSelfTest.java |    33 +
 ...eSystemLoopbackEmbeddedDualSyncSelfTest.java |    33 +
 ...leSystemLoopbackEmbeddedPrimarySelfTest.java |    33 +
 ...SystemLoopbackEmbeddedSecondarySelfTest.java |    34 +
 ...SystemLoopbackExternalDualAsyncSelfTest.java |    33 +
 ...eSystemLoopbackExternalDualSyncSelfTest.java |    33 +
 ...leSystemLoopbackExternalPrimarySelfTest.java |    33 +
 ...SystemLoopbackExternalSecondarySelfTest.java |    34 +
 ...condaryFileSystemInitializationSelfTest.java |   217 +
 ...teHadoopFileSystemShmemAbstractSelfTest.java |    94 +
 ...ileSystemShmemEmbeddedDualAsyncSelfTest.java |    33 +
 ...FileSystemShmemEmbeddedDualSyncSelfTest.java |    33 +
 ...pFileSystemShmemEmbeddedPrimarySelfTest.java |    33 +
 ...ileSystemShmemEmbeddedSecondarySelfTest.java |    33 +
 ...ileSystemShmemExternalDualAsyncSelfTest.java |    33 +
 ...FileSystemShmemExternalDualSyncSelfTest.java |    33 +
 ...pFileSystemShmemExternalPrimarySelfTest.java |    33 +
 ...ileSystemShmemExternalSecondarySelfTest.java |    33 +
 .../collections/HadoopAbstractMapTest.java      |   175 +
 .../HadoopConcurrentHashMultimapSelftest.java   |   280 +
 .../collections/HadoopHashMapSelfTest.java      |   133 +
 .../collections/HadoopSkipListSelfTest.java     |   320 +
 .../streams/HadoopDataStreamSelfTest.java       |   153 +
 .../taskexecutor/HadoopExecutorServiceTest.java |   119 +
 .../HadoopExternalTaskExecutionSelfTest.java    |   232 +
 .../HadoopExternalCommunicationSelfTest.java    |   222 +
 .../impl/util/BasicUserNameMapperSelfTest.java  |   134 +
 .../util/ChainedUserNameMapperSelfTest.java     |   111 +
 .../util/KerberosUserNameMapperSelfTest.java    |   100 +
 .../collections/HadoopAbstractMapTest.java      |   172 -
 .../HadoopConcurrentHashMultimapSelftest.java   |   278 -
 .../collections/HadoopHashMapSelfTest.java      |   177 -
 .../collections/HadoopSkipListSelfTest.java     |   318 -
 .../streams/HadoopDataStreamSelfTest.java       |   150 -
 .../hadoop/state/HadoopGroupingTestState.java   |    40 +
 .../state/HadoopJobTrackerSelfTestState.java    |    45 +
 .../HadoopMapReduceEmbeddedSelfTestState.java   |    32 +
 .../HadoopTaskExecutionSelfTestValues.java      |    51 +
 .../taskexecutor/HadoopExecutorServiceTest.java |   118 -
 .../HadoopExternalTaskExecutionSelfTest.java    |   232 -
 .../HadoopExternalCommunicationSelfTest.java    |   220 -
 .../testsuites/IgniteHadoopTestSuite.java       |   123 +-
 .../IgniteIgfsLinuxAndMacOSTestSuite.java       |    25 +-
 modules/hibernate/pom.xml                       |     2 +-
 .../hibernate/CacheHibernateBlobStore.java      |     4 +-
 modules/ignored-tests/README.txt                |     4 +
 modules/ignored-tests/pom.xml                   |   241 +
 ...gniteIgnoredBinarySimpleMapperTestSuite.java |    41 +
 .../IgniteIgnoredBinaryTestSuite.java           |    42 +
 .../testsuites/IgniteIgnoredTestSuite.java      |    61 +
 .../apache/ignite/testsuites/package-info.java  |    22 +
 modules/indexing/pom.xml                        |     2 +-
 .../query/h2/GridH2ResultSetIterator.java       |    62 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   878 +-
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |    95 +-
 .../query/h2/opt/GridH2CollocationModel.java    |   783 +
 .../processors/query/h2/opt/GridH2Cursor.java   |    36 +-
 .../query/h2/opt/GridH2DefaultTableEngine.java  |    38 +
 .../query/h2/opt/GridH2IndexBase.java           |  1392 +-
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |    17 +-
 .../query/h2/opt/GridH2MetaTable.java           |   383 +
 .../query/h2/opt/GridH2QueryContext.java        |   612 +
 .../query/h2/opt/GridH2QueryType.java           |    49 +
 .../query/h2/opt/GridH2RetryException.java      |    32 +
 .../processors/query/h2/opt/GridH2Row.java      |    86 +-
 .../query/h2/opt/GridH2RowDescriptor.java       |    28 +-
 .../query/h2/opt/GridH2RowFactory.java          |   179 +
 .../processors/query/h2/opt/GridH2Table.java    |   372 +-
 .../query/h2/opt/GridH2TreeIndex.java           |   142 +-
 .../processors/query/h2/opt/GridH2Utils.java    |   133 -
 .../query/h2/opt/GridH2ValueCacheObject.java    |    22 +-
 .../query/h2/opt/GridLuceneIndex.java           |     7 +-
 .../processors/query/h2/sql/GridSqlAlias.java   |    12 +
 .../processors/query/h2/sql/GridSqlColumn.java  |    22 +-
 .../processors/query/h2/sql/GridSqlConst.java   |     5 +
 .../processors/query/h2/sql/GridSqlElement.java |    11 +
 .../processors/query/h2/sql/GridSqlJoin.java    |    17 +-
 .../query/h2/sql/GridSqlOperation.java          |     2 +-
 .../query/h2/sql/GridSqlOperationType.java      |     8 +-
 .../query/h2/sql/GridSqlQueryParser.java        |    97 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |   432 +-
 .../processors/query/h2/sql/GridSqlSelect.java  |     9 +-
 .../processors/query/h2/sql/GridSqlTable.java   |    70 +
 .../processors/query/h2/sql/GridSqlType.java    |     5 +
 .../query/h2/twostep/GridMapQueryExecutor.java  |   499 +-
 .../query/h2/twostep/GridMergeIndex.java        |    85 +-
 .../h2/twostep/GridMergeIndexUnsorted.java      |     6 +-
 .../query/h2/twostep/GridMergeTable.java        |     4 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |   426 +-
 .../query/h2/twostep/GridThreadLocalTable.java  |    68 +-
 .../query/h2/twostep/msg/GridH2Array.java       |     9 +-
 .../query/h2/twostep/msg/GridH2Boolean.java     |    10 +-
 .../query/h2/twostep/msg/GridH2Byte.java        |     9 +-
 .../query/h2/twostep/msg/GridH2Bytes.java       |    11 +-
 .../query/h2/twostep/msg/GridH2CacheObject.java |     9 +-
 .../query/h2/twostep/msg/GridH2Date.java        |     9 +-
 .../query/h2/twostep/msg/GridH2Decimal.java     |    11 +-
 .../query/h2/twostep/msg/GridH2Double.java      |     9 +-
 .../query/h2/twostep/msg/GridH2Float.java       |     9 +-
 .../query/h2/twostep/msg/GridH2Geometry.java    |    11 +-
 .../h2/twostep/msg/GridH2IndexRangeRequest.java |   208 +
 .../twostep/msg/GridH2IndexRangeResponse.java   |   279 +
 .../query/h2/twostep/msg/GridH2Integer.java     |    20 +-
 .../query/h2/twostep/msg/GridH2JavaObject.java  |    11 +-
 .../query/h2/twostep/msg/GridH2Long.java        |     9 +-
 .../query/h2/twostep/msg/GridH2Null.java        |    15 +-
 .../h2/twostep/msg/GridH2QueryRequest.java      |   433 +
 .../query/h2/twostep/msg/GridH2RowMessage.java  |   116 +
 .../query/h2/twostep/msg/GridH2RowRange.java    |   181 +
 .../h2/twostep/msg/GridH2RowRangeBounds.java    |   188 +
 .../query/h2/twostep/msg/GridH2Short.java       |     9 +-
 .../query/h2/twostep/msg/GridH2String.java      |     9 +-
 .../query/h2/twostep/msg/GridH2Time.java        |     9 +-
 .../query/h2/twostep/msg/GridH2Timestamp.java   |    11 +-
 .../query/h2/twostep/msg/GridH2Uuid.java        |     9 +-
 .../h2/twostep/msg/GridH2ValueMessage.java      |     2 +-
 .../twostep/msg/GridH2ValueMessageFactory.java  |    22 +-
 .../CacheAbstractQueryMetricsSelfTest.java      |     6 +-
 .../CacheBinaryKeyConcurrentQueryTest.java      |   298 +
 .../cache/CacheConfigurationP2PTestServer.java  |     2 -
 .../cache/CacheIndexingOffheapCleanupTest.java  |   178 +
 .../cache/CacheSqlQueryValueCopySelfTest.java   |   226 +
 .../ClientReconnectAfterClusterRestartTest.java |   225 +
 ...idCacheReduceQueryMultithreadedSelfTest.java |   168 -
 ...niteBinaryObjectLocalQueryArgumentsTest.java |    28 +
 ...aryObjectQueryArgumentsOffheapLocalTest.java |    28 +
 ...teBinaryObjectQueryArgumentsOffheapTest.java |    30 +
 .../IgniteBinaryObjectQueryArgumentsTest.java   |   472 +-
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |     2 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |    30 +-
 .../IgniteCacheCrossCacheJoinRandomTest.java    |   442 +
 ...acheDistributedJoinCollocatedAndNotTest.java |   365 +
 ...acheDistributedJoinCustomAffinityMapper.java |   262 +
 .../IgniteCacheDistributedJoinNoIndexTest.java  |   299 +
 ...ributedJoinPartitionedAndReplicatedTest.java |   487 +
 ...CacheDistributedJoinQueryConditionsTest.java |   624 +
 .../cache/IgniteCacheDistributedJoinTest.java   |   316 +
 ...PartitionedAndReplicatedCollocationTest.java |   399 +
 ...teCacheJoinPartitionedAndReplicatedTest.java |   316 +
 ...IgniteCacheJoinQueryWithAffinityKeyTest.java |   646 +
 ...eLockPartitionOnAffinityRunAbstractTest.java |   430 +
 ...PartitionOnAffinityRunAtomicCacheOpTest.java |   329 +
 ...niteCacheLockPartitionOnAffinityRunTest.java |   852 ++
 ...LockPartitionOnAffinityRunTxCacheOpTest.java |    33 +
 ...titionOnAffinityRunWithCollisionSpiTest.java |   204 +
 .../cache/IgniteCacheOffheapEvictQueryTest.java |     2 +-
 .../cache/IgniteCacheOffheapIndexScanTest.java  |   195 +
 ...IgniteCachePrimitiveFieldsQuerySelfTest.java |   134 +
 .../cache/IgniteCacheQueriesLoadTest1.java      |   604 +
 .../cache/IgniteCacheQueryIndexSelfTest.java    |     2 +-
 .../cache/IgniteCacheQueryLoadSelfTest.java     |    12 +-
 .../cache/IgniteCacheUnionDuplicatesTest.java   |   151 +
 .../cache/IgniteCrossCachesJoinsQueryTest.java  |  1641 +++
 ...niteCacheDistributedQueryCancelSelfTest.java |   176 +
 ...butedQueryStopOnCancelOrTimeoutSelfTest.java |   253 +
 ...cheQueryAbstractDistributedJoinSelfTest.java |   290 +
 .../IgniteCacheQueryNoRebalanceSelfTest.java    |    82 +
 ...QueryNodeRestartDistributedJoinSelfTest.java |   228 +
 .../IgniteCacheQueryNodeRestartSelfTest2.java   |   125 +-
 ...nCancelOrTimeoutDistributedJoinSelfTest.java |   138 +
 ...gniteCacheReplicatedFieldsQuerySelfTest.java |     6 +-
 .../IgniteCacheReplicatedQuerySelfTest.java     |    56 +-
 .../IgniteCacheLocalFieldsQuerySelfTest.java    |    16 +
 ...eCacheLocalQueryCancelOrTimeoutSelfTest.java |   158 +
 ...dCacheAbstractReduceFieldsQuerySelfTest.java |   420 -
 ...ridCacheReduceFieldsQueryAtomicSelfTest.java |    38 -
 ...GridCacheReduceFieldsQueryLocalSelfTest.java |    37 -
 ...cheReduceFieldsQueryPartitionedSelfTest.java |    59 -
 ...acheReduceFieldsQueryReplicatedSelfTest.java |    37 -
 .../query/IgniteSqlSchemaIndexingTest.java      |     5 +-
 .../query/IgniteSqlSplitterSelfTest.java        |  1236 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   156 +-
 .../query/h2/opt/GridH2TableSelfTest.java       |    10 +-
 .../h2/sql/AbstractH2CompareQueryTest.java      |   165 +-
 .../query/h2/sql/BaseH2CompareQueryTest.java    |     3 +-
 .../query/h2/sql/GridQueryParsingTest.java      |    30 +-
 .../H2CompareBigQueryDistributedJoinsTest.java  |    28 +
 .../query/h2/sql/H2CompareBigQueryTest.java     |   119 +-
 .../IgniteBinaryCacheQueryTestSuite.java        |     1 -
 .../IgniteBinaryCacheQueryTestSuite2.java       |     3 +
 ...narySimpleNameMapperCacheQueryTestSuite.java |     2 -
 .../IgniteCacheAffinityRunTestSuite.java        |    45 +
 .../IgniteCacheQuerySelfTestSuite.java          |    50 +-
 .../IgniteCacheQuerySelfTestSuite2.java         |    25 +-
 .../IgniteCacheQuerySelfTestSuite3.java         |    20 +
 .../IgniteCacheWithIndexingTestSuite.java       |     9 +-
 modules/jcl/pom.xml                             |     2 +-
 modules/jms11/pom.xml                           |     2 +-
 .../stream/jms11/IgniteJmsStreamerTest.java     |   206 +-
 .../jms11/IgniteJmsStreamerTestSuite.java       |     2 +-
 modules/jta/pom.xml                             |     2 +-
 ...titionedCacheJtaLookupClassNameSelfTest.java |     4 +-
 .../ignite/testsuites/IgniteJtaTestSuite.java   |     3 +-
 modules/kafka/pom.xml                           |     5 +-
 .../ignite/stream/kafka/KafkaStreamer.java      |     4 +-
 .../kafka/connect/IgniteSinkConnector.java      |     9 +
 .../kafka/connect/IgniteSourceConnector.java    |     9 +
 .../stream/kafka/connect/package-info.java      |    21 +
 .../serialization/CacheEventDeserializer.java   |     3 +-
 .../serialization/CacheEventSerializer.java     |     3 +-
 .../connect/serialization/package-info.java     |    21 +
 .../kafka/connect/IgniteSinkConnectorTest.java  |    15 +-
 .../connect/IgniteSourceConnectorTest.java      |    14 +-
 modules/log4j/pom.xml                           |     2 +-
 modules/log4j2/pom.xml                          |     2 +-
 modules/mesos/pom.xml                           |     9 +-
 .../apache/ignite/mesos/ClusterProperties.java  |    39 +
 .../apache/ignite/mesos/IgniteFramework.java    |    10 +-
 .../ignite/mesos/resource/JettyServer.java      |    16 +-
 modules/mqtt/pom.xml                            |     2 +-
 modules/osgi-karaf/pom.xml                      |     2 +-
 .../osgi-karaf/src/main/resources/features.xml  |    12 +-
 modules/osgi-paxlogging/pom.xml                 |     2 +-
 modules/osgi/pom.xml                            |     2 +-
 .../include/ignite/binary/binary_containers.h   |   197 +-
 .../include/ignite/binary/binary_raw_reader.h   |    31 +-
 .../include/ignite/binary/binary_raw_writer.h   |    13 +
 .../include/ignite/binary/binary_reader.h       |    19 +-
 .../include/ignite/binary/binary_writer.h       |    13 +
 .../ignite/impl/binary/binary_id_resolver.h     |    10 +-
 .../ignite/impl/binary/binary_reader_impl.h     |    24 +-
 .../ignite/impl/binary/binary_type_manager.h    |    14 +-
 .../include/ignite/impl/binary/binary_utils.h   |    87 +
 .../ignite/impl/binary/binary_writer_impl.h     |     2 +-
 .../src/impl/binary/binary_reader_impl.cpp      |    44 +-
 .../src/impl/interop/interop_input_stream.cpp   |    14 +-
 modules/platforms/cpp/common/Makefile.am        |     9 +-
 .../platforms/cpp/common/include/Makefile.am    |    19 +-
 .../common/include/ignite/common/big_integer.h  |   523 +
 .../cpp/common/include/ignite/common/bits.h     |   218 +
 .../common/include/ignite/common/concurrent.h   |    10 +-
 .../cpp/common/include/ignite/common/decimal.h  |   527 +
 .../include/ignite/common/default_allocator.h   |    95 +
 .../include/ignite/common/dynamic_size_array.h  |   415 +
 .../include/ignite/common/fixed_size_array.h    |   288 +
 .../cpp/common/include/ignite/common/utils.h    |   106 +-
 .../platforms/cpp/common/include/ignite/date.h  |     2 +-
 .../platforms/cpp/common/include/ignite/guid.h  |     4 +-
 .../cpp/common/include/ignite/ignite_error.h    |    34 +-
 .../cpp/common/include/ignite/timestamp.h       |     2 +-
 .../cpp/common/os/win/src/common/utils.cpp      |    41 +-
 .../cpp/common/project/vs/common.vcxproj        |    17 +-
 .../common/project/vs/common.vcxproj.filters    |    27 +
 .../cpp/common/src/common/big_integer.cpp       |   829 ++
 .../platforms/cpp/common/src/common/bits.cpp    |   233 +
 .../platforms/cpp/common/src/common/decimal.cpp |   275 +
 .../platforms/cpp/common/src/ignite_error.cpp   |    18 +-
 modules/platforms/cpp/configure.ac              |     2 +-
 modules/platforms/cpp/core-test/Makefile.am     |    11 +-
 .../cpp/core-test/config/cache-query.xml        |    32 +-
 .../cpp/core-test/project/vs/core-test.vcxproj  |    10 +-
 .../project/vs/core-test.vcxproj.filters        |    17 +-
 .../platforms/cpp/core-test/src/bits_test.cpp   |   124 +
 .../cpp/core-test/src/cache_query_test.cpp      |   665 +-
 .../cpp/core-test/src/decimal_test.cpp          |  1101 ++
 .../core-test/src/dynamic_size_array_test.cpp   |   360 +
 .../cpp/core-test/src/fixed_size_array_test.cpp |   208 +
 .../cpp/core-test/src/interop_memory_test.cpp   |     5 +-
 .../cpp/core-test/src/interop_test.cpp          |   148 +
 modules/platforms/cpp/core/Makefile.am          |     1 +
 modules/platforms/cpp/core/include/Makefile.am  |     1 +
 .../cpp/core/include/ignite/cache/cache.h       |   201 +-
 .../cpp/core/include/ignite/cache/cache_entry.h |    16 +-
 .../include/ignite/cache/query/query_argument.h |    33 +-
 .../include/ignite/cache/query/query_cursor.h   |    23 +-
 .../ignite/cache/query/query_fields_cursor.h    |    15 +-
 .../ignite/cache/query/query_fields_row.h       |    22 +-
 .../include/ignite/cache/query/query_scan.h     |    10 +-
 .../core/include/ignite/cache/query/query_sql.h |    70 +-
 .../ignite/cache/query/query_sql_fields.h       |   120 +-
 .../platforms/cpp/core/include/ignite/ignite.h  |    21 +-
 .../core/include/ignite/ignite_configuration.h  |     4 +-
 .../cpp/core/include/ignite/ignition.h          |     4 +-
 .../impl/binary/binary_type_updater_impl.h      |    12 +-
 .../core/include/ignite/impl/cache/cache_impl.h |    41 +-
 .../ignite/impl/cache/query/query_batch.h       |   148 +
 .../impl/cache/query/query_fields_row_impl.h    |    30 +-
 .../ignite/impl/cache/query/query_impl.h        |    30 +-
 .../include/ignite/impl/ignite_environment.h    |    49 +-
 .../ignite/impl/interop/interop_target.h        |    33 +-
 .../cpp/core/include/ignite/impl/operations.h   |    47 +-
 .../include/ignite/transactions/transaction.h   |    68 +-
 .../ignite/transactions/transaction_consts.h    |    84 +-
 .../ignite/transactions/transaction_metrics.h   |    13 +-
 .../include/ignite/transactions/transactions.h  |    36 +-
 modules/platforms/cpp/core/namespaces.dox       |     2 +-
 .../platforms/cpp/core/project/vs/core.vcxproj  |     2 +
 .../cpp/core/project/vs/core.vcxproj.filters    |     6 +
 modules/platforms/cpp/core/src/ignition.cpp     |    16 +-
 .../impl/binary/binary_type_updater_impl.cpp    |    13 +-
 .../cpp/core/src/impl/cache/cache_impl.cpp      |    62 +-
 .../core/src/impl/cache/query/query_batch.cpp   |    52 +
 .../core/src/impl/cache/query/query_impl.cpp    |   193 +-
 .../cpp/core/src/impl/ignite_environment.cpp    |    45 +-
 .../core/src/impl/interop/interop_target.cpp    |    70 +-
 .../src/impl/transactions/transactions_impl.cpp |   140 +-
 modules/platforms/cpp/cpp.dxg                   |     4 +-
 modules/platforms/cpp/examples/README.txt       |     1 +
 modules/platforms/cpp/examples/configure.ac     |     2 +-
 .../query-example/src/query_example.cpp         |    54 +
 .../cpp/jni/include/ignite/jni/exports.h        |   113 +-
 .../platforms/cpp/jni/include/ignite/jni/java.h |   277 +-
 .../cpp/jni/include/ignite/jni/utils.h          |    91 +-
 modules/platforms/cpp/jni/project/vs/module.def |    95 +-
 modules/platforms/cpp/jni/src/exports.cpp       |   381 +-
 modules/platforms/cpp/jni/src/java.cpp          |  1472 +-
 modules/platforms/cpp/odbc-test/Makefile.am     |    18 +-
 .../odbc-test/config/queries-test-noodbc.xml    |   103 +
 .../cpp/odbc-test/config/queries-test.xml       |    38 +-
 .../platforms/cpp/odbc-test/include/Makefile.am |     4 +-
 .../odbc-test/include/sql_test_suite_fixture.h  |   197 +
 .../cpp/odbc-test/include/test_utils.h          |    53 +
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |    17 +-
 .../project/vs/odbc-test.vcxproj.filters        |    51 +-
 .../cpp/odbc-test/src/api_robustness_test.cpp   |  1006 ++
 .../src/application_data_buffer_test.cpp        |   164 +-
 .../platforms/cpp/odbc-test/src/column_test.cpp |    10 +-
 .../cpp/odbc-test/src/configuration_test.cpp    |   254 +-
 .../cpp/odbc-test/src/queries_test.cpp          |   459 +-
 .../platforms/cpp/odbc-test/src/row_test.cpp    |     9 +-
 .../src/sql_aggregate_functions_test.cpp        |   249 +
 .../src/sql_date_time_functions_test.cpp        |   213 +
 .../src/sql_numeric_functions_test.cpp          |   309 +
 .../cpp/odbc-test/src/sql_operators_test.cpp    |   214 +
 .../cpp/odbc-test/src/sql_outer_join_test.cpp   |   498 +
 .../odbc-test/src/sql_string_functions_test.cpp |   354 +
 .../odbc-test/src/sql_system_functions_test.cpp |    47 +
 .../odbc-test/src/sql_test_suite_fixture.cpp    |   288 +
 .../cpp/odbc-test/src/sql_types_test.cpp        |    60 +
 .../src/sql_value_expressions_test.cpp          |    94 +
 .../platforms/cpp/odbc-test/src/test_utils.cpp  |    36 +
 modules/platforms/cpp/odbc/Makefile.am          |     6 +-
 modules/platforms/cpp/odbc/include/Makefile.am  |     3 +-
 .../platforms/cpp/odbc/include/ignite/odbc.h    |     5 +-
 .../ignite/odbc/app/application_data_buffer.h   |    69 +-
 .../odbc/include/ignite/odbc/app/parameter.h    |    27 +-
 .../cpp/odbc/include/ignite/odbc/common_types.h |    29 +-
 .../include/ignite/odbc/config/configuration.h  |   304 +-
 .../cpp/odbc/include/ignite/odbc/connection.h   |    47 +-
 .../cpp/odbc/include/ignite/odbc/decimal.h      |   137 -
 .../ignite/odbc/diagnostic/diagnostic_record.h  |     2 +-
 .../cpp/odbc/include/ignite/odbc/dsn_config.h   |    61 +
 .../cpp/odbc/include/ignite/odbc/message.h      |    35 +-
 .../cpp/odbc/include/ignite/odbc/parser.h       |     3 -
 .../odbc/include/ignite/odbc/protocol_version.h |   188 +
 .../cpp/odbc/include/ignite/odbc/result_page.h  |     3 -
 .../cpp/odbc/include/ignite/odbc/statement.h    |    58 +-
 .../include/ignite/odbc/system/odbc_constants.h |     4 -
 .../odbc/system/ui/dsn_configuration_window.h   |   160 +
 .../cpp/odbc/include/ignite/odbc/utility.h      |    12 +-
 .../cpp/odbc/install/ignite-odbc-install.ini    |     1 +
 .../cpp/odbc/install/install_amd64.cmd          |    22 +-
 .../platforms/cpp/odbc/install/install_x86.cmd  |    11 +-
 .../ignite/odbc/system/ui/custom_window.h       |   189 +
 .../win/include/ignite/odbc/system/ui/window.h  |   201 +
 .../odbc/os/win/src/system/ui/custom_window.cpp |   184 +
 .../src/system/ui/dsn_configuration_window.cpp  |   326 +
 .../cpp/odbc/os/win/src/system/ui/window.cpp    |   192 +
 .../cpp/odbc/os/win/src/system_dsn.cpp          |   218 +
 .../platforms/cpp/odbc/project/vs/module.def    |     1 -
 .../platforms/cpp/odbc/project/vs/odbc.vcxproj  |    29 +-
 .../cpp/odbc/project/vs/odbc.vcxproj.filters    |    42 +-
 .../odbc/src/app/application_data_buffer.cpp    |   506 +-
 .../platforms/cpp/odbc/src/app/parameter.cpp    |   130 +-
 modules/platforms/cpp/odbc/src/column.cpp       |    20 +-
 modules/platforms/cpp/odbc/src/common_types.cpp |     3 +
 .../cpp/odbc/src/config/configuration.cpp       |   321 +-
 .../cpp/odbc/src/config/connection_info.cpp     |   110 +-
 modules/platforms/cpp/odbc/src/connection.cpp   |    73 +-
 modules/platforms/cpp/odbc/src/decimal.cpp      |   135 -
 .../odbc/src/diagnostic/diagnostic_record.cpp   |    26 +-
 .../diagnostic/diagnostic_record_storage.cpp    |     2 +-
 modules/platforms/cpp/odbc/src/dsn_config.cpp   |   123 +
 modules/platforms/cpp/odbc/src/entry_points.cpp |    57 +-
 modules/platforms/cpp/odbc/src/odbc.cpp         |   326 +-
 .../platforms/cpp/odbc/src/protocol_version.cpp |   145 +
 .../platforms/cpp/odbc/src/query/data_query.cpp |     8 +-
 modules/platforms/cpp/odbc/src/statement.cpp    |   152 +-
 modules/platforms/cpp/odbc/src/utility.cpp      |    39 +-
 .../Apache.Ignite.AspNet.Tests.csproj           |    77 +
 .../Apache.Ignite.AspNet.Tests.snk              |   Bin 0 -> 596 bytes
 .../Apache.Ignite.AspNet.Tests/App.config       |    73 +
 .../ExpiryCacheHolderTest.cs                    |   492 +
 .../IgniteOutputCacheProviderTest.cs            |   172 +
 .../IgniteSessionStateItemCollectionTest.cs     |   267 +
 .../IgniteSessionStateStoreDataTest.cs          |   117 +
 .../IgniteSessionStateStoreProviderTest.cs      |   488 +
 .../Properties/AssemblyInfo.cs                  |    42 +
 .../Apache.Ignite.AspNet.Tests/packages.config  |    22 +
 .../Apache.Ignite.AspNet.csproj                 |    78 +
 .../Apache.Ignite.AspNet.nuspec                 |    54 +
 .../Apache.Ignite.AspNet.ruleset                |    12 +
 .../Apache.Ignite.AspNet.snk                    |   Bin 0 -> 596 bytes
 .../IgniteOutputCacheProvider.cs                |   124 +
 .../IgniteSessionStateStoreProvider.cs          |   509 +
 .../Apache.Ignite.AspNet/IgniteWebUtils.cs      |    44 +
 .../Apache.Ignite.AspNet/Impl/ConfigUtil.cs     |   115 +
 .../Impl/ExpiryCacheHolder.cs                   |   113 +
 .../Impl/IgniteSessionStateItemCollection.cs    |   534 +
 .../Impl/IgniteSessionStateStoreData.cs         |   116 +
 .../Impl/SessionStateLockResult.cs              |    91 +
 .../dotnet/Apache.Ignite.AspNet/Package-Info.cs |    26 +
 .../Properties/AssemblyInfo.cs                  |    42 +
 .../Apache.Ignite.Benchmarks/BenchmarkRunner.cs |     5 +-
 .../Interop/PlatformBenchmarkBase.cs            |     2 +-
 .../Properties/AssemblyInfo.cs                  |    40 +-
 .../Apache.Ignite.Core.Tests.NuGet.csproj       |    60 +-
 .../AspNetTest.cs                               |    73 +
 .../EntityFrameworkCacheTest.cs                 |    62 +
 .../Log4NetTest.cs                              |    87 +
 .../Apache.Ignite.Core.Tests.NuGet/NLogTest.cs  |    82 +
 .../Properties/AssemblyInfo.cs                  |    36 +-
 .../SchemaTest.cs                               |    62 +
 .../install-package.ps1                         |    21 +-
 .../packages.config                             |    15 +-
 .../Apache.Ignite.Core.Tests.TestDll.csproj     |     5 -
 .../Properties/AssemblyInfo.cs                  |    40 +-
 .../Apache.Ignite.Core.Tests.csproj             |    74 +-
 .../Binary/BinaryBuilderSelfTest.cs             |   406 +-
 .../Binary/BinaryReaderWriterTest.cs            |   171 +
 .../Binary/BinarySelfTest.cs                    |   167 +-
 .../Binary/BinaryStringTest.cs                  |   100 +
 .../Binary/IO/BinaryStreamsTest.cs              |   151 +
 .../Binary/JavaTypeMappingTest.cs               |   158 +
 .../Binary/TypeResolverTest.cs                  |   107 +
 .../Cache/Affinity/AffinityFieldTest.cs         |   199 +
 .../Affinity/AffinityFunctionSpringTest.cs      |   184 +
 .../Cache/Affinity/AffinityFunctionTest.cs      |   514 +
 .../Cache/Affinity/AffinityKeyTest.cs           |    66 +
 .../Cache/Affinity/AffinityTest.cs              |   138 +
 .../Affinity/AffinityTopologyVersionTest.cs     |    59 +
 .../Cache/CacheAbstractTest.cs                  |   119 +-
 .../Cache/CacheAffinityFieldTest.cs             |   199 -
 .../Cache/CacheAffinityTest.cs                  |   139 -
 .../Cache/CacheConfigurationTest.cs             |   134 +-
 .../Cache/CacheResultTest.cs                    |    75 +
 .../Cache/CacheSwapSpaceTest.cs                 |   120 +
 .../Cache/Query/CacheLinqTest.cs                |   297 +-
 .../Query/CacheQueriesCodeConfigurationTest.cs  |    12 +
 .../Cache/Query/CacheQueriesTest.cs             |   292 +-
 .../Continuous/ContinuousQueryAbstractTest.cs   |    60 +-
 .../Cache/Store/CacheParallelLoadStoreTest.cs   |     2 +-
 .../Cache/Store/CacheStoreAdapterTest.cs        |    90 +
 .../Cache/Store/CacheStoreTest.cs               |    50 +-
 .../Cache/Store/CacheTestParallelLoadStore.cs   |     9 +
 .../Cache/Store/CacheTestStore.cs               |    50 +-
 .../Collections/MultiValueDictionaryTest.cs     |    58 +
 .../Collections/ReadOnlyCollectionTest.cs       |    59 +
 .../Collections/ReadOnlyDictionaryTest.cs       |    70 +
 .../Common/IgniteGuidTest.cs                    |    62 +
 .../Compute/AbstractTaskTest.cs                 |    49 +-
 .../Compute/BinarizableClosureTaskTest.cs       |    18 +-
 .../Compute/BinarizableTaskTest.cs              |    31 +-
 .../Compute/CancellationTest.cs                 |    10 +
 .../Compute/ClosureTaskTest.cs                  |   192 +-
 .../Compute/ComputeApiTest.cs                   |   220 +-
 .../Compute/FailoverTaskSelfTest.cs             |     7 +-
 .../Compute/IgniteExceptionTaskSelfTest.cs      |    55 +-
 .../Compute/ResourceTaskTest.cs                 |    59 +-
 .../Compute/SerializableClosureTaskTest.cs      |     5 +
 .../Compute/TaskAdapterTest.cs                  |    21 +-
 .../Compute/TaskResultTest.cs                   |    24 +-
 .../Config/Cache/Affinity/affinity-function.xml |   127 +
 .../Cache/Affinity/affinity-function2.xml       |    49 +
 .../Config/Compute/compute-standalone.xml       |     1 +
 .../Config/Log/custom-log.xml                   |    50 +
 .../Config/Log/dotnet-log4j.xml                 |   143 +
 .../Config/cache-query-continuous.xml           |     1 +
 .../Config/cache-query.xml                      |    38 +-
 .../native-client-test-cache-affinity.xml       |     2 +-
 .../ConsoleRedirectTest.cs                      |   177 +
 .../Apache.Ignite.Core.Tests/DeploymentTest.cs  |     2 +-
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |   196 +-
 .../Examples/Example.cs                         |     8 +-
 .../Examples/ExamplesTest.cs                    |    48 +-
 .../Examples/PathUtil.cs                        |    10 +-
 .../Examples/ProjectFilesTest.cs                |     2 +-
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |   128 +-
 .../IgniteConfigurationSectionTest.cs           |    52 +
 .../IgniteConfigurationSerializerTest.cs        |   288 +-
 .../IgniteConfigurationTest.cs                  |    60 +-
 .../IgniteStartStopTest.cs                      |     3 +-
 .../Apache.Ignite.Core.Tests/LifecycleTest.cs   |     9 +
 .../Log/CustomLoggerTest.cs                     |   449 +
 .../Log/DefaultLoggerTest.cs                    |   114 +
 .../Log/Log4NetLoggerTest.cs                    |   188 +
 .../Log/NLogLoggerTest.cs                       |   166 +
 .../Apache.Ignite.Core.Tests/MessagingTest.cs   |    16 +-
 .../ProjectFilesTest.cs                         |    98 +-
 .../Properties/AssemblyInfo.cs                  |    40 +-
 .../Apache.Ignite.Core.Tests/ReconnectTest.cs   |    94 +-
 .../Services/ServiceProxyTest.cs                |     6 +-
 .../Services/ServicesTest.cs                    |    28 +-
 .../Apache.Ignite.Core.Tests/TestAppConfig.cs   |   100 +
 .../Apache.Ignite.Core.Tests/TestRunner.cs      |    76 +-
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |    56 +-
 .../TypeResolverTest.cs                         |   107 -
 .../dotnet/Apache.Ignite.Core.Tests/app.config  |     3 +-
 .../Apache.Ignite.Core.Tests/custom_app.config  |    35 +
 .../Apache.Ignite.Core.Tests/packages.config    |    24 +
 .../Apache.Ignite.Core.Schema.nuspec            |    52 +
 .../Apache.Ignite.Core.csproj                   |    48 +-
 .../Apache.Ignite.Core.nuspec                   |     3 -
 .../Binary/BinaryConfiguration.cs               |     5 +-
 .../Binary/BinaryReflectiveSerializer.cs        |   193 +-
 .../Binary/IBinaryObjectBuilder.cs              |     2 +-
 .../Cache/Affinity/AffinityFunctionContext.cs   |   120 +
 .../Cache/Affinity/AffinityKey.cs               |    11 +
 .../Cache/Affinity/AffinityTopologyVersion.cs   |   138 +
 .../Cache/Affinity/Fair/FairAffinityFunction.cs |    33 +
 .../Cache/Affinity/Fair/Package-Info.cs         |    26 +
 .../Cache/Affinity/IAffinityFunction.cs         |    82 +
 .../Cache/Affinity/Package-Info.cs              |    26 +
 .../Cache/Affinity/Rendezvous/Package-Info.cs   |    26 +
 .../Rendezvous/RendezvousAffinityFunction.cs    |    32 +
 .../Cache/CachePartialUpdateException.cs        |     6 +-
 .../Cache/Configuration/CacheConfiguration.cs   |    37 +-
 .../Cache/Configuration/Package-Info.cs         |    26 +
 .../Cache/Configuration/QueryEntity.cs          |    25 +-
 .../Cache/Configuration/QueryField.cs           |    19 +-
 .../Cache/Eviction/Package-Info.cs              |    26 +
 .../Apache.Ignite.Core/Cache/ICacheLock.cs      |     2 +-
 .../Apache.Ignite.Core/Cache/ICacheMetrics.cs   |     8 +-
 .../Cache/Query/SqlFieldsQuery.cs               |    43 +-
 .../Apache.Ignite.Core/Cache/Query/SqlQuery.cs  |    13 +
 .../Store/CacheParallelLoadStoreAdapter.cs      |     7 +
 .../Apache.Ignite.Core/Cluster/IClusterGroup.cs |    20 +
 .../Apache.Ignite.Core/Common/JavaException.cs  |    66 +
 .../Communication/Package-Info.cs               |    26 +
 .../Communication/Tcp/Package-Info.cs           |    26 +
 .../Compute/ComputeTaskSplitAdapter.cs          |     2 +-
 .../Apache.Ignite.Core/Compute/ICompute.cs      |     6 +-
 .../Configuration/AtomicConfiguration.cs        |     9 +
 .../Configuration/Package-Info.cs               |    26 +
 .../Discovery/Package-Info.cs                   |    26 +
 .../Discovery/Tcp/Multicast/Package-Info.cs     |    26 +
 .../Multicast/TcpDiscoveryMulticastIpFinder.cs  |     7 +
 .../Discovery/Tcp/Package-Info.cs               |    26 +
 .../Discovery/Tcp/Static/Package-Info.cs        |    26 +
 .../Apache.Ignite.Core/Events/DiscoveryEvent.cs |     4 +-
 .../Apache.Ignite.Core/Events/EventBase.cs      |     2 +-
 .../Apache.Ignite.Core/Events/EventReader.cs    |    12 +-
 .../dotnet/Apache.Ignite.Core/IIgnite.cs        |    33 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |   130 +-
 .../IgniteConfigurationSection.xsd              |  1200 +-
 .../dotnet/Apache.Ignite.Core/Ignition.cs       |   168 +-
 .../Impl/Binary/BinarizableSerializer.cs        |    22 +-
 .../Apache.Ignite.Core/Impl/Binary/Binary.cs    |     2 +-
 .../Impl/Binary/BinaryFullTypeDescriptor.cs     |     8 +-
 .../Impl/Binary/BinaryObject.cs                 |     2 +
 .../Impl/Binary/BinaryObjectBuilder.cs          |    18 +-
 .../Impl/Binary/BinaryObjectHandle.cs           |    59 -
 .../Impl/Binary/BinaryObjectHeader.cs           |     7 +-
 .../Impl/Binary/BinaryObjectSchemaSerializer.cs |     2 +-
 .../Impl/Binary/BinaryProcessor.cs              |   156 +
 .../Impl/Binary/BinaryReader.cs                 |    58 +-
 .../Impl/Binary/BinaryReaderExtensions.cs       |    16 +-
 .../Impl/Binary/BinaryReflectiveActions.cs      |    10 +-
 .../BinaryReflectiveSerializerInternal.cs       |   169 +
 .../Binary/BinarySurrogateTypeDescriptor.cs     |     6 +-
 .../Impl/Binary/BinarySystemTypeSerializer.cs   |    20 +-
 .../Impl/Binary/BinaryUtils.cs                  |   192 +-
 .../Impl/Binary/BinaryWriter.cs                 |   161 +-
 .../Impl/Binary/DateTimeHolder.cs               |     4 +-
 .../Impl/Binary/DateTimeSerializer.cs           |    48 +
 .../Impl/Binary/IBinarySerializerInternal.cs    |    42 +
 .../Impl/Binary/IBinarySystemTypeSerializer.cs  |    34 -
 .../Impl/Binary/IBinaryTypeDescriptor.cs        |     2 +-
 .../Impl/Binary/Io/BinaryHeapStream.cs          |     2 +-
 .../Impl/Binary/Io/BinaryStreamAdapter.cs       |     5 +
 .../Impl/Binary/Io/BinaryStreamBase.cs          |    25 +-
 .../Impl/Binary/Io/IBinaryStream.cs             |     2 +-
 .../Apache.Ignite.Core/Impl/Binary/JavaTypes.cs |    75 +-
 .../Impl/Binary/Marshaller.cs                   |   163 +-
 .../Impl/Binary/SerializableObjectHolder.cs     |     6 +-
 .../Impl/Binary/SerializableSerializer.cs       |    48 +
 .../Binary/Structure/BinaryStructureTracker.cs  |     7 +-
 .../Impl/Binary/UserSerializerProxy.cs          |    68 +
 .../Impl/Cache/Affinity/AffinityFunctionBase.cs |   140 +
 .../Affinity/AffinityFunctionSerializer.cs      |   277 +
 .../Cache/Affinity/PlatformAffinityFunction.cs  |    74 +
 .../Impl/Cache/CacheAffinityImpl.cs             |     5 +-
 .../Impl/Cache/CacheEntryFilterHolder.cs        |    10 +-
 .../Impl/Cache/CacheEntryProcessorHolder.cs     |     8 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |   702 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheLock.cs  |    24 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheOp.cs    |    44 +-
 .../Cache/Event/JavaCacheEntryEventFilter.cs    |     2 +
 .../Impl/Cache/ICacheInternal.cs                |    14 +
 .../Impl/Cache/ICacheLockInternal.cs            |    47 +
 .../Impl/Cache/Query/AbstractQueryCursor.cs     |    10 +-
 .../Continuous/ContinuousQueryFilterHolder.cs   |     8 +-
 .../Continuous/ContinuousQueryHandleImpl.cs     |   117 +-
 .../Impl/Cache/Query/FieldsQueryCursor.cs       |     3 +
 .../Impl/Cache/Store/CacheStore.cs              |   112 +-
 .../Impl/Cluster/ClusterGroupImpl.cs            |   134 +-
 .../Impl/Cluster/IClusterGroupEx.cs             |    35 -
 .../Impl/Collections/MultiValueDictionary.cs    |    26 -
 .../Impl/Collections/ReadOnlyDictionary.cs      |     2 +-
 .../Apache.Ignite.Core/Impl/Common/Classpath.cs |    12 +-
 .../Impl/Common/DelegateConverter.cs            |    44 +
 .../Apache.Ignite.Core/Impl/Common/Fnv1Hash.cs  |     2 +-
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |    30 +-
 .../Common/IgniteConfigurationXmlSerializer.cs  |    21 +-
 .../Impl/Common/IgniteHome.cs                   |    30 +-
 .../Apache.Ignite.Core/Impl/Common/Logger.cs    |    37 -
 .../Impl/Common/ObjectInfoHolder.cs             |    86 +
 .../Impl/Common/ResizeableArray.cs              |    64 -
 .../Impl/Compute/Closure/ComputeActionJob.cs    |     6 +-
 .../Impl/Compute/Closure/ComputeFuncJob.cs      |     8 +-
 .../Impl/Compute/Closure/ComputeOutFuncJob.cs   |     9 +-
 .../Impl/Compute/ComputeFunc.cs                 |    12 +-
 .../Impl/Compute/ComputeImpl.cs                 |    46 +-
 .../Impl/Compute/ComputeJob.cs                  |    16 +-
 .../Impl/Compute/ComputeJobHolder.cs            |     8 +-
 .../Impl/Compute/ComputeOutFunc.cs              |    11 +-
 .../Impl/Compute/ComputeTaskHolder.cs           |     3 +-
 .../Impl/DataStructures/AtomicLong.cs           |    34 +-
 .../Impl/DataStructures/AtomicReference.cs      |     8 +-
 .../Impl/DataStructures/AtomicSequence.cs       |    26 +-
 .../Impl/Datastream/DataStreamerImpl.cs         |    49 +-
 .../Impl/Datastream/StreamReceiverHolder.cs     |     2 +-
 .../Impl/Events/EventTypeConverter.cs           |     3 +
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |   183 +-
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |    37 +-
 .../Apache.Ignite.Core/Impl/Handle/Handle.cs    |    12 +-
 .../Impl/Handle/HandleRegistry.cs               |    11 +-
 .../Apache.Ignite.Core/Impl/Handle/IHandle.cs   |     5 -
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |    98 +-
 .../Apache.Ignite.Core/Impl/IgniteManager.cs    |     6 +-
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |    64 +-
 .../Apache.Ignite.Core/Impl/IgniteUtils.cs      |    64 +-
 .../Impl/InteropExceptionHolder.cs              |    88 -
 .../Impl/LifecycleBeanHolder.cs                 |     2 +-
 .../Apache.Ignite.Core/Impl/Log/JavaLogger.cs   |   110 +
 .../Memory/PlatformBigEndianMemoryStream.cs     |    34 +-
 .../Impl/Memory/PlatformMemoryStream.cs         |     3 +-
 .../Impl/Memory/PlatformMemoryUtils.cs          |    30 -
 .../Impl/Memory/PlatformRawMemory.cs            |     5 +
 .../Impl/Messaging/MessageListenerHolder.cs     |    14 +-
 .../Impl/Messaging/Messaging.cs                 |   113 +-
 .../Apache.Ignite.Core/Impl/NativeMethods.cs    |     6 +
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |   348 +-
 .../Impl/Services/ServiceProxyInvoker.cs        |    14 +-
 .../Impl/Services/Services.cs                   |   165 +-
 .../Impl/SwapSpace/SwapSpaceSerializer.cs       |    99 +
 .../Impl/Transactions/TransactionsImpl.cs       |    64 +-
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |   290 +-
 .../Impl/Unmanaged/UnmanagedCallbackHandlers.cs |     9 +
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |   313 +-
 .../Impl/Unmanaged/UnmanagedTarget.cs           |     2 +-
 .../Impl/Unmanaged/UnmanagedUtils.cs            |   584 +-
 .../Apache.Ignite.Core/Interop/Package-Info.cs  |    26 +
 .../Lifecycle/ClientReconnectEventArgs.cs       |    47 +
 .../Apache.Ignite.Core/Log/CategoryLogger.cs    |    82 +
 .../dotnet/Apache.Ignite.Core/Log/ILogger.cs    |    51 +
 .../dotnet/Apache.Ignite.Core/Log/LogLevel.cs   |    53 +
 .../Apache.Ignite.Core/Log/LoggerExtensions.cs  |   320 +
 .../NuGet/LINQPad/BinaryModeExample.linq        |    94 +
 .../NuGet/LINQPad/PutGetExample.linq            |     2 +-
 .../NuGet/LINQPad/QueryExample.linq             |     6 +-
 .../Properties/AssemblyInfo.cs                  |    43 +-
 .../SwapSpace/File/FileSwapSpaceSpi.cs          |   105 +
 .../SwapSpace/ISwapSpaceSpi.cs                  |    35 +
 .../Transactions/TransactionConfiguration.cs    |    12 +
 .../TransactionDeadlockException.cs             |    71 +
 .../Apache.Ignite.EntityFramework.Tests.csproj  |    96 +
 .../Apache.Ignite.EntityFramework.Tests.snk     |   Bin 0 -> 596 bytes
 .../App.config                                  |    71 +
 .../ArrayDbDataReaderTests.cs                   |   192 +
 .../DbCachingPolicyTest.cs                      |    43 +
 .../EntityFrameworkCacheInitializationTest.cs   |   137 +
 .../EntityFrameworkCacheTest.cs                 |   946 ++
 .../Properties/AssemblyInfo.cs                  |    39 +
 .../packages.config                             |    23 +
 .../Apache.Ignite.EntityFramework.csproj        |    95 +
 .../Apache.Ignite.EntityFramework.nuspec        |    57 +
 .../Apache.Ignite.EntityFramework.snk           |   Bin 0 -> 596 bytes
 .../DbCachingMode.cs                            |    48 +
 .../DbCachingPolicy.cs                          |    72 +
 .../DbQueryInfo.cs                              |    78 +
 .../IDbCachingPolicy.cs                         |    58 +
 .../IgniteDbConfiguration.cs                    |   240 +
 .../Impl/ArrayDbDataReader.cs                   |   305 +
 .../Impl/DataReaderField.cs                     |    74 +
 .../Impl/DataReaderResult.cs                    |    93 +
 .../Impl/DbCache.cs                             |   295 +
 .../Impl/DbCacheKey.cs                          |    92 +
 .../Impl/DbCommandDefinitionProxy.cs            |    51 +
 .../Impl/DbCommandInfo.cs                       |   158 +
 .../Impl/DbCommandProxy.cs                      |   263 +
 .../Impl/DbProviderServicesProxy.cs             |   169 +
 .../Impl/DbTransactionInterceptor.cs            |   134 +
 .../Properties/AssemblyInfo.cs                  |    41 +
 .../packages.config                             |    20 +
 modules/platforms/dotnet/Apache.Ignite.FxCop    |     2 +
 .../Apache.Ignite.Linq.csproj                   |    11 +-
 .../Apache.Ignite.Linq.nuspec                   |     7 +-
 .../Apache.Ignite.Linq/CacheExtensions.cs       |    24 +-
 .../dotnet/Apache.Ignite.Linq/CompiledQuery.cs  |     1 +
 .../dotnet/Apache.Ignite.Linq/CompiledQuery2.cs |   257 +
 .../Apache.Ignite.Linq/ICacheQueryable.cs       |    32 +-
 .../Apache.Ignite.Linq/Impl/AliasDictionary.cs  |    74 +-
 .../Impl/CacheFieldsQueryExecutor.cs            |   147 +-
 .../Impl/CacheQueryExpressionVisitor.cs         |    19 +-
 .../Impl/CacheQueryModelVisitor.cs              |    21 +-
 .../Apache.Ignite.Linq/Impl/CacheQueryable.cs   |    10 +-
 .../Impl/CacheQueryableBase.cs                  |    27 +-
 .../Apache.Ignite.Linq/Impl/ExpressionWalker.cs |    44 +-
 .../Apache.Ignite.Linq/Impl/ICacheQueryProxy.cs |    40 -
 .../Impl/ICacheQueryableInternal.cs             |    14 +-
 .../NuGet/LINQPad/QueryExample.linq             |     4 +-
 .../dotnet/Apache.Ignite.Linq/Package-Info.cs   |    26 +
 .../Properties/AssemblyInfo.cs                  |    40 +-
 .../dotnet/Apache.Ignite.Linq/QueryOptions.cs   |    91 +
 .../Apache.Ignite.Log4Net.csproj                |    76 +
 .../Apache.Ignite.Log4Net.nuspec                |    50 +
 .../Apache.Ignite.Log4Net.snk                   |   Bin 0 -> 596 bytes
 .../IgniteLog4NetLogger.cs                      |   123 +
 .../Properties/AssemblyInfo.cs                  |    40 +
 .../Apache.Ignite.Log4Net/packages.config       |    20 +
 .../Apache.Ignite.NLog.csproj                   |    69 +
 .../Apache.Ignite.NLog.nuspec                   |    50 +
 .../Apache.Ignite.NLog/Apache.Ignite.NLog.snk   |   Bin 0 -> 596 bytes
 .../Apache.Ignite.NLog/IgniteNLogLogger.cs      |   125 +
 .../Properties/AssemblyInfo.cs                  |    40 +
 .../dotnet/Apache.Ignite.NLog/packages.config   |    22 +
 modules/platforms/dotnet/Apache.Ignite.dxg      |     4 +-
 modules/platforms/dotnet/Apache.Ignite.sln      |    85 +-
 .../dotnet/Apache.Ignite.sln.DotSettings        |    10 +-
 .../Apache.Ignite.sln.TeamCity.DotSettings      |     1 +
 .../dotnet/Apache.Ignite/Apache.Ignite.csproj   |     5 -
 .../dotnet/Apache.Ignite/Config/Configurator.cs |    11 +-
 .../dotnet/Apache.Ignite/IgniteRunner.cs        |    20 +-
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |    40 +-
 modules/platforms/dotnet/README.txt             |    13 +-
 .../dotnet/examples/Apache.Ignite.Examples.sln  |     8 -
 .../Apache.Ignite.Examples.csproj               |     2 +
 .../examples/Apache.Ignite.Examples/App.config  |    34 +-
 .../Compute/ClosureExample.cs                   |    16 +-
 .../Compute/TaskExample.cs                      |    18 +-
 .../Datagrid/BinaryModeExample.cs               |   272 +
 .../Datagrid/ContinuousQueryExample.cs          |    20 +-
 .../Datagrid/DataStreamerExample.cs             |    16 +-
 .../Datagrid/LinqExample.cs                     |   232 +-
 .../Datagrid/OptimisticTransactionExample.cs    |   110 +
 .../Datagrid/PutGetExample.cs                   |    16 +-
 .../Datagrid/QueryExample.cs                    |   231 +-
 .../Datagrid/StoreExample.cs                    |    24 +-
 .../Datagrid/TransactionExample.cs              |    18 +-
 .../Events/EventsExample.cs                     |    22 +-
 .../Messaging/MessagingExample.cs               |    14 +-
 .../Misc/LifecycleExample.cs                    |    26 +-
 .../Properties/AssemblyInfo.cs                  |    44 +-
 .../Services/IMapService.cs                     |     4 +-
 .../Services/ServicesExample.cs                 |    14 +-
 .../Apache.Ignite.ExamplesDll.csproj            |     2 +-
 .../Apache.Ignite.ExamplesDll/Binary/Account.cs |     6 +-
 .../Apache.Ignite.ExamplesDll/Binary/Address.cs |     2 +-
 .../Binary/Employee.cs                          |    21 +-
 .../Binary/EmployeeKey.cs                       |    88 -
 .../Binary/Organization.cs                      |     2 +-
 .../Compute/AverageSalaryJob.cs                 |    14 +-
 .../Compute/AverageSalaryTask.cs                |     9 +-
 .../Compute/CharacterCountClosure.cs            |     6 +-
 .../Compute/CharacterCountReducer.cs            |     4 +-
 .../Datagrid/ContinuousQueryFilter.cs           |     6 +-
 .../Datagrid/EmployeeStore.cs                   |    13 +-
 .../Datagrid/EmployeeStorePredicate.cs          |     5 +-
 .../Datagrid/ScanQueryFilter.cs                 |    50 +
 .../Events/LocalListener.cs                     |    10 +-
 .../Messaging/LocalListener.cs                  |     8 +-
 .../Messaging/RemoteOrderedListener.cs          |    10 +-
 .../Messaging/RemoteUnorderedListener.cs        |    10 +-
 .../Properties/AssemblyInfo.cs                  |    42 +-
 .../Services/MapService.cs                      |    12 +-
 .../dotnet/examples/Config/examples-config.xml  |    98 -
 modules/rest-http/pom.xml                       |    27 +-
 .../http/jetty/GridJettyJsonConfig.java         |   317 -
 .../http/jetty/GridJettyObjectMapper.java       |   274 +
 .../http/jetty/GridJettyRestHandler.java        |   132 +-
 modules/scalar-2.10/pom.xml                     |     2 +-
 modules/scalar/pom.xml                          |     2 +-
 modules/schedule/pom.xml                        |     2 +-
 .../processors/schedule/ScheduleFutureImpl.java |     7 +-
 .../schedule/GridScheduleSelfTest.java          |    33 +-
 modules/schema-import-db/pom.xml                |    17 +-
 .../apache/ignite/schema/parser/DbColumn.java   |    16 +-
 .../parser/dialect/DatabaseMetadataDialect.java |    27 +
 .../parser/dialect/JdbcMetadataDialect.java     |    26 +-
 .../parser/dialect/MySQLMetadataDialect.java    |    24 +-
 .../parser/dialect/OracleMetadataDialect.java   |    10 +-
 modules/schema-import/README.txt                |    12 +-
 modules/schema-import/pom.xml                   |    10 +-
 .../ignite/schema/generator/CodeGenerator.java  |    23 +-
 .../ignite/schema/model/PojoDescriptor.java     |    24 +-
 .../schema/parser/DatabaseMetadataParser.java   |     4 +-
 .../ignite/schema/ui/SchemaImportApp.java       |     3 +-
 .../schema/test/AbstractSchemaImportTest.java   |    27 +-
 .../test/generator/CodeGeneratorTest.java       |    28 +-
 .../schema/test/generator/XmlGeneratorTest.java |    22 +-
 .../ignite/schema/test/model/CacheConfig.txt    |   409 +
 .../schema/test/model/ignite-type-metadata.xml  |    50 +-
 .../test/parser/DbMetadataParserTest.java       |     6 +-
 modules/slf4j/pom.xml                           |     2 +-
 .../apache/ignite/logger/slf4j/Slf4jLogger.java |     2 +-
 modules/spark-2.10/pom.xml                      |     2 +-
 modules/spark/pom.xml                           |     2 +-
 .../org/apache/ignite/spark/IgniteContext.scala |    12 +-
 .../org/apache/ignite/spark/IgniteRDD.scala     |   128 +-
 .../apache/ignite/spark/JavaIgniteContext.scala |     6 +-
 .../org/apache/ignite/spark/JavaIgniteRDD.scala |    11 +
 .../ignite/spark/impl/IgniteAbstractRDD.scala   |    17 +-
 .../apache/ignite/spark/impl/IgniteSqlRDD.scala |     7 +-
 .../spark/impl/JavaIgniteAbstractRDD.scala      |    34 -
 .../spark/JavaStandaloneIgniteRDDSelfTest.java  |     9 +-
 .../ignite/spark/EntityTestAllTypeFields.scala  |     7 +-
 .../org/apache/ignite/spark/IgniteRDDSpec.scala |   145 +-
 modules/spring/pom.xml                          |     9 +-
 .../org/apache/ignite/IgniteSpringBean.java     |    23 +-
 .../apache/ignite/cache/spring/SpringCache.java |    44 +-
 .../ignite/cache/spring/SpringCacheManager.java |    22 +-
 .../GridResourceSpringBeanInjector.java         |    39 +-
 .../spring/SpringTransactionManager.java        |    59 +-
 .../src/test/config/jdbc-pojo-store-builtin.xml |    12 +-
 .../src/test/config/jdbc-pojo-store-obj.xml     |    12 +-
 .../test/java/config/spring-transactions.xml    |     4 +-
 .../ignite/TestInjectionLifecycleBean.java      |    42 +
 .../spring/GridSpringCacheManagerSelfTest.java  |   438 +
 .../cache/spring/GridSpringCacheTestKey.java    |    61 +
 .../spring/GridSpringCacheTestKeyGenerator.java |    40 +
 .../spring/GridSpringCacheTestService.java      |   181 +
 .../GridSpringDynamicCacheTestService.java      |    85 +
 .../SpringCacheManagerContextInjectionTest.java |   125 +
 .../ignite/cache/spring/spring-caching.xml      |    57 +
 .../jdbc/CacheJdbcBlobStoreFactorySelfTest.java |    12 +-
 .../jdbc/CacheJdbcPojoStoreFactorySelfTest.java |    25 +-
 .../store/jdbc/CachePojoStoreXmlSelfTest.java   |    13 +-
 .../CachePojoStoreXmlWithSqlEscapeSelfTest.java |    28 +
 .../GridSpringResourceInjectionSelfTest.java    |   311 +-
 .../GridTransformSpringInjectionSelfTest.java   |   186 +
 .../spring-resource-with-duplicate-beans.xml    |    30 +
 .../processors/resource/spring-resource.xml     |     2 +-
 .../p2p/GridP2PUserVersionChangeSelfTest.java   |     7 +-
 .../org/apache/ignite/spring-injection-test.xml |    43 +
 .../spring/GridSpringCacheManagerSelfTest.java  |   342 -
 .../ignite/spring/GridSpringCacheTestKey.java   |    61 -
 .../spring/GridSpringCacheTestKeyGenerator.java |    40 -
 .../spring/GridSpringCacheTestService.java      |   125 -
 .../GridSpringDynamicCacheTestService.java      |    85 -
 .../org/apache/ignite/spring/spring-caching.xml |    57 -
 .../testsuites/IgniteResourceSelfTestSuite.java |    11 +-
 .../testsuites/IgniteSpringTestSuite.java       |    30 +-
 ...gTransactionManagerContextInjectionTest.java |   125 +
 modules/ssh/pom.xml                             |     2 +-
 modules/storm/README.txt                        |     2 +-
 modules/storm/pom.xml                           |     4 +-
 .../ignite/stream/storm/StormStreamer.java      |    10 +-
 .../storm/StormIgniteStreamerSelfTest.java      |    20 +-
 .../ignite/stream/storm/TestStormSpout.java     |    10 +-
 modules/tools/pom.xml                           |     2 +-
 modules/twitter/pom.xml                         |     2 +-
 modules/urideploy/pom.xml                       |     2 +-
 .../spi/deployment/uri/UriDeploymentSpi.java    |     2 +-
 .../scanners/http/UriDeploymentHttpScanner.java |     8 +-
 modules/visor-console-2.10/pom.xml              |     2 +-
 modules/visor-console/pom.xml                   |     2 +-
 .../ignite/visor/commands/VisorConsole.scala    |     6 +-
 .../commands/alert/VisorAlertCommand.scala      |    45 +-
 .../commands/cache/VisorCacheCommand.scala      |    89 +-
 .../commands/cache/VisorCacheResetCommand.scala |   129 +
 .../commands/cache/VisorCacheStopCommand.scala  |     7 +-
 .../config/VisorConfigurationCommand.scala      |    23 +-
 .../commands/events/VisorEventsCommand.scala    |    29 +-
 .../visor/commands/gc/VisorGcCommand.scala      |    11 +-
 .../visor/commands/kill/VisorKillCommand.scala  |   184 +-
 .../visor/commands/node/VisorNodeCommand.scala  |    13 +-
 .../visor/commands/vvm/VisorVvmCommand.scala    |    11 +-
 .../scala/org/apache/ignite/visor/visor.scala   |    78 +-
 .../cache/VisorCacheResetCommandSpec.scala      |   114 +
 modules/visor-plugins/pom.xml                   |     2 +-
 modules/web-agent/.gitignore                    |     2 -
 modules/web-agent/README.txt                    |    87 -
 .../web-agent/assembly/release-web-agent.xml    |    74 -
 modules/web-agent/bin/ignite-web-agent.bat      |    70 -
 modules/web-agent/bin/ignite-web-agent.sh       |    87 -
 modules/web-agent/demo/README.txt               |     4 -
 modules/web-agent/demo/db-init.sql              |   102 -
 modules/web-agent/jdbc-drivers/README.txt       |    10 -
 modules/web-agent/logs/README.txt               |     5 -
 modules/web-agent/pom.xml                       |   169 -
 .../console/agent/AgentConfiguration.java       |   255 -
 .../ignite/console/agent/AgentLauncher.java     |   337 -
 .../apache/ignite/console/agent/AgentUtils.java |   111 -
 .../console/agent/handlers/AbstractHandler.java |   110 -
 .../console/agent/handlers/DatabaseHandler.java |   298 -
 .../console/agent/handlers/RestHandler.java     |   276 -
 .../ignite/console/demo/AgentClusterDemo.java   |   614 -
 .../ignite/console/demo/AgentMetadataDemo.java  |    92 -
 .../apache/ignite/console/demo/model/Car.java   |   152 -
 .../ignite/console/demo/model/Country.java      |   152 -
 .../ignite/console/demo/model/Department.java   |   152 -
 .../ignite/console/demo/model/Employee.java     |   356 -
 .../ignite/console/demo/model/Parking.java      |   152 -
 .../src/main/resources/log4j.properties         |    53 -
 modules/web-console/.gitignore                  |     6 +
 modules/web-console/DEVNOTES.txt                |    28 +-
 modules/web-console/README.txt                  |     4 +-
 modules/web-console/backend/.eslintrc           |   186 +
 modules/web-console/backend/.gitignore          |     6 +
 .../web-console/backend/agent_dists/README.txt  |     7 +
 modules/web-console/backend/app/agent.js        |   803 +
 modules/web-console/backend/app/app.js          |    63 +
 modules/web-console/backend/app/browser.js      |   443 +
 modules/web-console/backend/app/configure.js    |    86 +
 modules/web-console/backend/app/mongo.js        |   867 ++
 modules/web-console/backend/app/mongoose.js     |    29 +
 modules/web-console/backend/app/nconf.js        |    48 +
 modules/web-console/backend/app/routes.js       |    64 +
 modules/web-console/backend/app/settings.js     |    78 +
 .../backend/config/settings.json.sample         |    30 +
 .../backend/errors/AppErrorException.js         |    36 +
 .../backend/errors/AuthFailedException.js       |    30 +
 .../backend/errors/DuplicateKeyException.js     |    28 +
 .../backend/errors/IllegalAccessError.js        |    29 +
 .../backend/errors/IllegalArgumentException.js  |    29 +
 .../backend/errors/MissingResourceException.js  |    30 +
 .../backend/errors/ServerErrorException.js      |    36 +
 modules/web-console/backend/errors/index.js     |    41 +
 modules/web-console/backend/index.js            |   104 +
 modules/web-console/backend/injector.js         |    30 +
 modules/web-console/backend/middlewares/api.js  |    44 +
 modules/web-console/backend/middlewares/host.js |    39 +
 modules/web-console/backend/middlewares/user.js |    36 +
 modules/web-console/backend/package.json        |    62 +
 modules/web-console/backend/routes/admin.js     |    84 +
 modules/web-console/backend/routes/agent.js     |    53 +
 modules/web-console/backend/routes/caches.js    |    65 +
 modules/web-console/backend/routes/clusters.js  |    64 +
 .../web-console/backend/routes/configuration.js |    41 +
 modules/web-console/backend/routes/demo.js      |   133 +
 .../web-console/backend/routes/demo/caches.json |    87 +
 .../backend/routes/demo/clusters.json           |    50 +
 .../backend/routes/demo/domains.json            |   312 +
 .../web-console/backend/routes/demo/igfss.json  |    10 +
 modules/web-console/backend/routes/domains.js   |    76 +
 modules/web-console/backend/routes/igfss.js     |    65 +
 modules/web-console/backend/routes/notebooks.js |    80 +
 modules/web-console/backend/routes/profile.js   |    73 +
 modules/web-console/backend/routes/public.js    |   129 +
 modules/web-console/backend/services/agents.js  |    83 +
 modules/web-console/backend/services/auth.js    |   118 +
 modules/web-console/backend/services/caches.js  |   152 +
 .../web-console/backend/services/clusters.js    |   148 +
 .../backend/services/configurations.js          |    59 +
 modules/web-console/backend/services/domains.js |   195 +
 modules/web-console/backend/services/igfss.js   |   144 +
 modules/web-console/backend/services/mails.js   |   131 +
 .../web-console/backend/services/notebooks.js   |   104 +
 .../web-console/backend/services/sessions.js    |    65 +
 modules/web-console/backend/services/spaces.js  |    75 +
 modules/web-console/backend/services/users.js   |   229 +
 modules/web-console/backend/test/app/db.js      |    66 +
 .../web-console/backend/test/app/httpAgent.js   |    50 +
 .../web-console/backend/test/app/mockgoose.js   |    30 +
 .../backend/test/config/settings.json           |    20 +
 .../web-console/backend/test/data/accounts.json |    19 +
 .../web-console/backend/test/data/caches.json   |    97 +
 .../web-console/backend/test/data/clusters.json |    54 +
 .../web-console/backend/test/data/domains.json  |   317 +
 .../web-console/backend/test/data/igfss.json    |    12 +
 .../web-console/backend/test/data/spaces.json   |    14 +
 modules/web-console/backend/test/index.js       |    35 +
 modules/web-console/backend/test/injector.js    |    48 +
 .../web-console/backend/test/routes/clusters.js |    83 +
 .../web-console/backend/test/routes/public.js   |    68 +
 .../backend/test/unit/AuthService.test.js       |   105 +
 .../backend/test/unit/CacheService.test.js      |   147 +
 .../backend/test/unit/ClusterService.test.js    |   147 +
 .../backend/test/unit/DomainService.test.js     |   167 +
 .../backend/test/unit/IgfsService.test.js       |   145 +
 .../docker/compose/backend/.dockerignore        |     1 +
 .../docker/compose/backend/Dockerfile           |    30 +
 .../web-console/docker/compose/backend/build.sh |    57 +
 .../docker/compose/docker-compose.yml           |    59 +
 .../docker/compose/frontend/.dockerignore       |     3 +
 .../docker/compose/frontend/Dockerfile          |    32 +
 .../docker/compose/frontend/DockerfileBuild     |    30 +
 .../docker/compose/frontend/build.sh            |    59 +
 .../docker/compose/frontend/nginx/nginx.conf    |    57 +
 .../compose/frontend/nginx/web-console.conf     |    59 +
 .../web-console/docker/standalone/.dockerignore |     2 +
 .../web-console/docker/standalone/Dockerfile    |    87 +
 modules/web-console/docker/standalone/build.sh  |    59 +
 .../docker/standalone/docker-compose.yml        |    41 +
 .../web-console/docker/standalone/entrypoint.sh |    23 +
 .../docker/standalone/nginx/nginx.conf          |    55 +
 .../docker/standalone/nginx/web-console.conf    |    54 +
 modules/web-console/frontend/.babelrc           |     9 +
 modules/web-console/frontend/.eslintrc          |   198 +
 modules/web-console/frontend/.gitignore         |     7 +
 modules/web-console/frontend/app/app.config.js  |    96 +
 modules/web-console/frontend/app/app.js         |   272 +
 .../frontend/app/controllers/auth.controller.js |    30 +
 .../controllers/reset-password.controller.js    |    50 +
 .../web-console/frontend/app/data/colors.json   |    22 +
 .../frontend/app/data/countries.json            |    94 +
 .../frontend/app/data/demo-info.json            |    14 +
 .../web-console/frontend/app/data/dialects.json |     9 +
 .../frontend/app/data/event-types.json          |   169 +
 .../frontend/app/data/getting-started.json      |   109 +
 .../frontend/app/data/java-classes.json         |    21 +
 .../frontend/app/data/java-keywords.json        |    55 +
 .../frontend/app/data/java-primitives.json      |     9 +
 .../frontend/app/data/jdbc-types.json           |    44 +
 .../frontend/app/data/pom-dependencies.json     |    20 +
 .../frontend/app/data/sql-keywords.json         |    41 +
 .../frontend/app/decorator/select.js            |    77 +
 .../frontend/app/decorator/tooltip.js           |    73 +
 .../app/directives/auto-focus.directive.js      |    26 +
 .../app/directives/bs-affix-update.directive.js |    34 +
 .../app/directives/centered/centered.css        |    37 +
 .../directives/centered/centered.directive.js   |    26 +
 .../directives/copy-to-clipboard.directive.js   |    29 +
 .../hide-on-state-change.directive.js           |    31 +
 .../information/information.directive.js        |    30 +
 .../app/directives/information/information.jade |    20 +
 .../app/directives/information/information.scss |    56 +
 .../frontend/app/directives/match.directive.js  |    27 +
 .../app/directives/on-click-focus.directive.js  |    26 +
 .../directives/on-enter-focus-move.directive.js |    29 +
 .../app/directives/on-enter.directive.js        |    32 +
 .../app/directives/on-escape.directive.js       |    32 +
 .../app/directives/on-focus-out.directive.js    |    37 +
 .../directives/restore-input-focus.directive.js |    24 +
 .../directives/retain-selection.directive.js    |    67 +
 .../ui-ace-docker/ui-ace-docker.controller.js   |    33 +
 .../ui-ace-docker/ui-ace-docker.directive.js    |    46 +
 .../directives/ui-ace-docker/ui-ace-docker.jade |    31 +
 .../ui-ace-java/ui-ace-java.controller.js       |    92 +
 .../ui-ace-java/ui-ace-java.directive.js        |    63 +
 .../app/directives/ui-ace-java/ui-ace-java.jade |    22 +
 .../ui-ace-pojos/ui-ace-pojos.controller.js     |    95 +
 .../ui-ace-pojos/ui-ace-pojos.directive.js      |    46 +
 .../directives/ui-ace-pojos/ui-ace-pojos.jade   |    40 +
 .../ui-ace-pom/ui-ace-pom.controller.js         |    33 +
 .../ui-ace-pom/ui-ace-pom.directive.js          |    41 +
 .../app/directives/ui-ace-pom/ui-ace-pom.jade   |    17 +
 .../ui-ace-sharp/ui-ace-sharp.controller.js     |    32 +
 .../ui-ace-sharp/ui-ace-sharp.directive.js      |   133 +
 .../directives/ui-ace-sharp/ui-ace-sharp.jade   |    22 +
 .../ui-ace-spring/ui-ace-spring.controller.js   |    88 +
 .../ui-ace-spring/ui-ace-spring.directive.js    |    66 +
 .../directives/ui-ace-spring/ui-ace-spring.jade |    17 +
 .../app/directives/ui-ace-tabs.directive.js     |    24 +
 .../ui-grid-settings/ui-grid-settings.jade      |    33 +
 .../ui-grid-settings/ui-grid-settings.scss      |    38 +
 .../frontend/app/filters/byName.filter.js       |    23 +
 .../frontend/app/filters/default-name.filter.js |    21 +
 .../app/filters/domainsValidation.filter.js     |    33 +
 .../frontend/app/filters/duration.filter.js     |    38 +
 .../frontend/app/filters/hasPojo.filter.js      |    21 +
 .../frontend/app/helpers/jade/form.jade         |    28 +
 .../helpers/jade/form/form-field-checkbox.jade  |    38 +
 .../helpers/jade/form/form-field-datalist.jade  |    51 +
 .../app/helpers/jade/form/form-field-down.jade  |    18 +
 .../helpers/jade/form/form-field-dropdown.jade  |    50 +
 .../helpers/jade/form/form-field-feedback.jade  |    32 +
 .../app/helpers/jade/form/form-field-label.jade |    23 +
 .../helpers/jade/form/form-field-number.jade    |    52 +
 .../helpers/jade/form/form-field-password.jade  |    47 +
 .../app/helpers/jade/form/form-field-text.jade  |    47 +
 .../app/helpers/jade/form/form-field-up.jade    |    18 +
 .../app/helpers/jade/form/form-group.jade       |    23 +
 .../frontend/app/helpers/jade/mixins.jade       |   555 +
 .../frontend/app/modules/Demo/Demo.module.js    |   166 +
 .../frontend/app/modules/ace.module.js          |   269 +
 .../frontend/app/modules/agent/agent.module.js  |   343 +
 .../app/modules/branding/branding.module.js     |    45 +
 .../app/modules/branding/branding.provider.js   |   111 +
 .../app/modules/branding/features.directive.js  |    35 +
 .../app/modules/branding/footer.directive.js    |    34 +
 .../modules/branding/header-logo.directive.js   |    34 +
 .../app/modules/branding/header-logo.jade       |    18 +
 .../modules/branding/header-title.directive.js  |    35 +
 .../branding/powered-by-apache.directive.js     |    35 +
 .../app/modules/branding/powered-by-apache.jade |    18 +
 .../app/modules/branding/terms.directive.js     |    30 +
 .../configuration/EventGroups.provider.js       |    30 +
 .../modules/configuration/Sidebar.provider.js   |    39 +
 .../modules/configuration/Version.service.js    |    95 +
 .../configuration/configuration.module.js       |    65 +
 .../generator/AbstractTransformer.js            |   341 +
 .../modules/configuration/generator/Beans.js    |   379 +
 .../generator/ConfigurationGenerator.js         |  1785 +++
 .../configuration/generator/Docker.service.js   |    78 +
 .../generator/JavaTransformer.service.js        |  1721 +++
 .../generator/PlatformGenerator.js              |   522 +
 .../configuration/generator/Pom.service.js      |   233 +
 .../generator/Properties.service.js             |    74 +
 .../configuration/generator/Readme.service.js   |    79 +
 .../generator/SharpTransformer.service.js       |   243 +
 .../generator/SpringTransformer.service.js      |   325 +
 .../configuration/generator/StringBuilder.js    |    76 +
 .../defaults/cache.platform.provider.js         |    60 +
 .../generator/defaults/cache.provider.js        |   129 +
 .../defaults/cluster.platform.provider.js       |    49 +
 .../generator/defaults/cluster.provider.js      |   293 +
 .../generator/defaults/igfs.provider.js         |    68 +
 .../configuration/generator/generator-common.js |   625 +
 .../configuration/generator/generator-java.js   |  3617 +++++
 .../generator/generator-optional.js             |    25 +
 .../configuration/generator/generator-spring.js |  2111 +++
 .../modules/configuration/sidebar.directive.js  |    30 +
 .../modules/dialog/dialog-content.directive.js  |    31 +
 .../modules/dialog/dialog-title.directive.js    |    31 +
 .../app/modules/dialog/dialog.controller.js     |    40 +
 .../app/modules/dialog/dialog.directive.js      |    32 +
 .../app/modules/dialog/dialog.factory.js        |    32 +
 .../frontend/app/modules/dialog/dialog.jade     |    26 +
 .../app/modules/dialog/dialog.module.js         |    32 +
 .../field/bs-select-placeholder.directive.js    |    47 +
 .../app/modules/form/field/down.directive.js    |    39 +
 .../app/modules/form/field/feedback.scss        |    37 +
 .../frontend/app/modules/form/field/field.scss  |    43 +
 .../field/form-control-feedback.directive.js    |    40 +
 .../form/field/input/autofocus.directive.js     |    30 +
 .../app/modules/form/field/input/text.scss      |    40 +
 .../app/modules/form/field/label.directive.js   |    47 +
 .../app/modules/form/field/tooltip.directive.js |    49 +
 .../app/modules/form/field/up.directive.js      |    39 +
 .../frontend/app/modules/form/form.module.js    |    94 +
 .../app/modules/form/group/add.directive.js     |    40 +
 .../app/modules/form/group/tooltip.directive.js |    40 +
 .../app/modules/form/panel/chevron.directive.js |    53 +
 .../app/modules/form/panel/field.directive.js   |    69 +
 .../app/modules/form/panel/panel.directive.js   |    37 +
 .../app/modules/form/panel/revert.directive.js  |    54 +
 .../modules/form/services/FormGUID.service.js   |    22 +
 .../form/validator/ipaddress.directive.js       |    86 +
 .../validator/java-built-in-class.directive.js  |    35 +
 .../form/validator/java-identifier.directive.js |    35 +
 .../form/validator/java-keywords.directive.js   |    39 +
 .../validator/java-package-name.directive.js    |    31 +
 .../java-package-specified.directive.js         |    39 +
 .../form/validator/property-unique.directive.js |    47 +
 .../property-value-specified.directive.js       |    31 +
 .../modules/form/validator/unique.directive.js  |    49 +
 .../modules/form/validator/uuid.directive.js    |    31 +
 .../getting-started/GettingStarted.provider.js  |   112 +
 .../frontend/app/modules/loading/loading.css    |    73 +
 .../app/modules/loading/loading.directive.js    |    51 +
 .../frontend/app/modules/loading/loading.jade   |    23 +
 .../app/modules/loading/loading.module.js       |    26 +
 .../app/modules/loading/loading.service.js      |    48 +
 .../app/modules/navbar/Navbar.provider.js       |    28 +
 .../app/modules/navbar/Userbar.provider.js      |    28 +
 .../app/modules/navbar/navbar.directive.js      |    30 +
 .../app/modules/navbar/navbar.module.js         |    33 +
 .../app/modules/navbar/userbar.directive.js     |    48 +
 .../frontend/app/modules/nodes/Nodes.service.js |    69 +
 .../modules/nodes/nodes-dialog.controller.js    |    68 +
 .../app/modules/nodes/nodes-dialog.jade         |    35 +
 .../app/modules/nodes/nodes-dialog.scss         |    37 +
 .../frontend/app/modules/nodes/nodes.module.js  |    27 +
 .../frontend/app/modules/socket.module.js       |    41 +
 .../frontend/app/modules/sql/Notebook.data.js   |   165 +
 .../app/modules/sql/Notebook.service.js         |    74 +
 .../app/modules/sql/notebook.controller.js      |    60 +
 .../app/modules/sql/scan-filter-input.jade      |    39 +
 .../modules/sql/scan-filter-input.service.js    |    51 +
 .../frontend/app/modules/sql/sql.controller.js  |  1719 +++
 .../frontend/app/modules/sql/sql.module.js      |    60 +
 .../frontend/app/modules/states/admin.state.js  |    35 +
 .../app/modules/states/configuration.state.js   |    97 +
 .../configuration/Configuration.resource.js     |    42 +
 .../configuration/caches/client-near-cache.jade |    50 +
 .../configuration/caches/concurrency.jade       |    65 +
 .../states/configuration/caches/general.jade    |    69 +
 .../states/configuration/caches/memory.jade     |   109 +
 .../configuration/caches/near-cache-client.jade |    51 +
 .../configuration/caches/near-cache-server.jade |    52 +
 .../configuration/caches/node-filter.jade       |    59 +
 .../states/configuration/caches/query.jade      |   111 +
 .../states/configuration/caches/rebalance.jade  |    66 +
 .../states/configuration/caches/statistics.jade |    39 +
 .../states/configuration/caches/store.jade      |   232 +
 .../states/configuration/clusters/atomic.jade   |    54 +
 .../configuration/clusters/attributes.jade      |    57 +
 .../states/configuration/clusters/binary.jade   |    77 +
 .../configuration/clusters/cache-key-cfg.jade   |    50 +
 .../configuration/clusters/checkpoint.jade      |    85 +
 .../configuration/clusters/checkpoint/fs.jade   |    66 +
 .../configuration/clusters/checkpoint/jdbc.jade |    45 +
 .../configuration/clusters/checkpoint/s3.jade   |   174 +
 .../configuration/clusters/collision.jade       |    63 +
 .../clusters/collision/custom.jade              |    24 +
 .../clusters/collision/fifo-queue.jade          |    27 +
 .../clusters/collision/job-stealing.jade        |    63 +
 .../clusters/collision/priority-queue.jade      |    42 +
 .../configuration/clusters/communication.jade   |   100 +
 .../configuration/clusters/connector.jade       |   104 +
 .../configuration/clusters/deployment.jade      |   114 +
 .../configuration/clusters/discovery.jade       |    88 +
 .../states/configuration/clusters/events.jade   |    68 +
 .../states/configuration/clusters/failover.jade |    73 +
 .../states/configuration/clusters/general.jade  |    76 +
 .../clusters/general/discovery/cloud.jade       |   134 +
 .../clusters/general/discovery/google.jade      |    38 +
 .../clusters/general/discovery/jdbc.jade        |    32 +
 .../clusters/general/discovery/multicast.jade   |    99 +
 .../clusters/general/discovery/s3.jade          |    27 +
 .../clusters/general/discovery/shared.jade      |    23 +
 .../clusters/general/discovery/vm.jade          |    79 +
 .../clusters/general/discovery/zookeeper.jade   |    85 +
 .../bounded-exponential-backoff.jade            |    27 +
 .../discovery/zookeeper/retrypolicy/custom.jade |    24 +
 .../retrypolicy/exponential-backoff.jade        |    27 +
 .../zookeeper/retrypolicy/forever.jade          |    22 +
 .../zookeeper/retrypolicy/n-times.jade          |    25 +
 .../zookeeper/retrypolicy/one-time.jade         |    23 +
 .../zookeeper/retrypolicy/until-elapsed.jade    |    25 +
 .../states/configuration/clusters/igfs.jade     |    38 +
 .../configuration/clusters/load-balancing.jade  |   104 +
 .../states/configuration/clusters/logger.jade   |    66 +
 .../configuration/clusters/logger/custom.jade   |    25 +
 .../configuration/clusters/logger/log4j.jade    |    50 +
 .../configuration/clusters/logger/log4j2.jade   |    39 +
 .../configuration/clusters/marshaller.jade      |    76 +
 .../states/configuration/clusters/metrics.jade  |    51 +
 .../states/configuration/clusters/odbc.jade     |    48 +
 .../states/configuration/clusters/ssl.jade      |   110 +
 .../states/configuration/clusters/swap.jade     |    72 +
 .../states/configuration/clusters/thread.jade   |    48 +
 .../states/configuration/clusters/time.jade     |    47 +
 .../configuration/clusters/transactions.jade    |    69 +
 .../states/configuration/domains/general.jade   |    52 +
 .../states/configuration/domains/query.jade     |   172 +
 .../states/configuration/domains/store.jade     |   127 +
 .../modules/states/configuration/igfs/dual.jade |    42 +
 .../states/configuration/igfs/fragmentizer.jade |    43 +
 .../states/configuration/igfs/general.jade      |    57 +
 .../modules/states/configuration/igfs/ipc.jade  |    60 +
 .../modules/states/configuration/igfs/misc.jade |   108 +
 .../states/configuration/igfs/secondary.jade    |    45 +
 .../configuration/preview-panel.directive.js    |   239 +
 .../summary/summary-tabs.directive.js           |    50 +
 .../configuration/summary/summary.controller.js |   401 +
 .../frontend/app/modules/states/errors.state.js |    43 +
 .../frontend/app/modules/states/logout.state.js |    35 +
 .../app/modules/states/password.state.js        |    46 +
 .../app/modules/states/profile.state.js         |    35 +
 .../frontend/app/modules/states/signin.state.js |    43 +
 .../app/modules/user/AclRoute.provider.js       |    47 +
 .../frontend/app/modules/user/Auth.service.js   |    56 +
 .../frontend/app/modules/user/User.service.js   |    51 +
 .../frontend/app/modules/user/permissions.js    |    28 +
 .../frontend/app/modules/user/user.module.js    |    73 +
 .../app/services/ChartColors.service.js         |    22 +
 .../frontend/app/services/Clone.service.js      |    64 +
 .../frontend/app/services/Confirm.service.js    |    68 +
 .../app/services/ConfirmBatch.service.js        |    92 +
 .../app/services/CopyToClipboard.service.js     |    50 +
 .../frontend/app/services/Countries.service.js  |    31 +
 .../app/services/ErrorPopover.service.js        |   129 +
 .../frontend/app/services/Focus.service.js      |    33 +
 .../frontend/app/services/FormUtils.service.js  |   439 +
 .../app/services/InetAddress.service.js         |    53 +
 .../frontend/app/services/JavaTypes.service.js  |   182 +
 .../app/services/LegacyTable.service.js         |   229 +
 .../app/services/LegacyUtils.service.js         |   548 +
 .../frontend/app/services/Messages.service.js   |    63 +
 .../app/services/ModelNormalizer.service.js     |    59 +
 .../frontend/app/services/SqlTypes.service.js   |    65 +
 .../app/services/UnsavedChangesGuard.service.js |    38 +
 modules/web-console/frontend/app/vendor.js      |    56 +
 .../frontend/controllers/admin-controller.js    |    93 +
 .../frontend/controllers/caches-controller.js   |   619 +
 .../frontend/controllers/clusters-controller.js |   822 ++
 .../frontend/controllers/domains-controller.js  |  1858 +++
 .../frontend/controllers/igfs-controller.js     |   416 +
 .../frontend/controllers/profile-controller.js  |    95 +
 .../frontend/gulpfile.babel.js/index.js         |    26 +
 .../frontend/gulpfile.babel.js/paths.js         |    83 +
 .../frontend/gulpfile.babel.js/tasks/build.js   |    21 +
 .../frontend/gulpfile.babel.js/tasks/bundle.js  |    32 +
 .../frontend/gulpfile.babel.js/tasks/clean.js   |    32 +
 .../frontend/gulpfile.babel.js/tasks/copy.js    |    33 +
 .../gulpfile.babel.js/tasks/ignite-modules.js   |    55 +
 .../frontend/gulpfile.babel.js/tasks/jade.js    |    49 +
 .../frontend/gulpfile.babel.js/tasks/test.js    |    92 +
 .../frontend/gulpfile.babel.js/tasks/watch.js   |    33 +
 .../gulpfile.babel.js/webpack/common.js         |   189 +
 .../webpack/environments/development.js         |    78 +
 .../webpack/environments/production.js          |    45 +
 .../webpack/environments/test.js                |    52 +
 .../frontend/gulpfile.babel.js/webpack/index.js |    34 +
 .../webpack/plugins/progress.js                 |    82 +
 .../frontend/ignite_modules/README.txt          |     6 +
 .../frontend/ignite_modules/index.js            |    27 +
 modules/web-console/frontend/package.json       |   125 +
 modules/web-console/frontend/public/favicon.ico |   Bin 0 -> 1150 bytes
 .../frontend/public/images/cache.png            |   Bin 0 -> 23700 bytes
 .../frontend/public/images/cluster.png          |   Bin 0 -> 29376 bytes
 .../frontend/public/images/docker.png           |   Bin 0 -> 521 bytes
 .../frontend/public/images/domains.png          |   Bin 0 -> 23828 bytes
 .../web-console/frontend/public/images/igfs.png |   Bin 0 -> 14307 bytes
 .../frontend/public/images/ignite-logo.png      |   Bin 0 -> 1982 bytes
 .../frontend/public/images/ignite-logo@2x.png   |   Bin 0 -> 3325 bytes
 .../frontend/public/images/ignite-puzzle.png    |   Bin 0 -> 71974 bytes
 .../web-console/frontend/public/images/java.png |   Bin 0 -> 170 bytes
 .../frontend/public/images/pb-ignite.png        |   Bin 0 -> 3493 bytes
 .../frontend/public/images/pb-ignite@2x.png     |   Bin 0 -> 8558 bytes
 .../frontend/public/images/query-chart.png      |   Bin 0 -> 16637 bytes
 .../frontend/public/images/query-metadata.png   |   Bin 0 -> 32298 bytes
 .../frontend/public/images/query-table.png      |   Bin 0 -> 29189 bytes
 .../frontend/public/images/summary.png          |   Bin 0 -> 31997 bytes
 .../web-console/frontend/public/images/xml.png  |   Bin 0 -> 232 bytes
 .../public/stylesheets/_bootstrap-custom.scss   |    65 +
 .../stylesheets/_bootstrap-variables.scss       |   891 ++
 .../stylesheets/_font-awesome-custom.scss       |    32 +
 .../public/stylesheets/blocks/error.scss        |    31 +
 .../frontend/public/stylesheets/form-field.scss |   108 +
 .../frontend/public/stylesheets/style.scss      |  2229 +++
 .../frontend/public/stylesheets/variables.scss  |    28 +
 .../frontend/test/e2e/exampe.test.js            |    40 +
 .../frontend/test/karma.conf.babel.js           |    91 +
 modules/web-console/frontend/test/karma.conf.js |    19 +
 .../frontend/test/protractor.conf.js            |    50 +
 .../frontend/test/unit/JavaTransformer.test.js  |    57 +
 .../frontend/test/unit/JavaTypes.test.js        |   128 +
 .../frontend/test/unit/SharpTransformer.test.js |    55 +
 .../test/unit/SpringTransformer.test.js         |    57 +
 .../frontend/test/unit/SqlTypes.test.js         |    51 +
 .../frontend/test/unit/UserAuth.test.js         |    35 +
 .../frontend/test/unit/Version.test.js          |    82 +
 .../test/unit/defaultName.filter.test.js        |    38 +
 modules/web-console/frontend/views/403.jade     |    22 +
 modules/web-console/frontend/views/404.jade     |    22 +
 modules/web-console/frontend/views/base.jade    |    22 +
 .../frontend/views/configuration/caches.jade    |    54 +
 .../frontend/views/configuration/clusters.jade  |    68 +
 .../views/configuration/domains-import.jade     |   167 +
 .../frontend/views/configuration/domains.jade   |    66 +
 .../frontend/views/configuration/igfs.jade      |    51 +
 .../frontend/views/configuration/sidebar.jade   |    29 +
 .../summary-project-structure.jade              |    27 +
 .../views/configuration/summary-tabs.jade       |    25 +
 .../frontend/views/configuration/summary.jade   |    83 +
 .../frontend/views/includes/footer.jade         |    23 +
 .../frontend/views/includes/header.jade         |    51 +
 modules/web-console/frontend/views/index.jade   |    47 +
 modules/web-console/frontend/views/reset.jade   |    48 +
 .../frontend/views/settings/admin.jade          |    76 +
 .../frontend/views/settings/profile.jade        |    76 +
 modules/web-console/frontend/views/signin.jade  |   163 +
 .../frontend/views/sql/cache-metadata.jade      |    40 +
 .../frontend/views/sql/chart-settings.jade      |    40 +
 .../frontend/views/sql/notebook-new.jade        |    33 +
 .../frontend/views/sql/paragraph-rate.jade      |    31 +
 modules/web-console/frontend/views/sql/sql.jade |   229 +
 .../views/templates/agent-download.jade         |    50 +
 .../frontend/views/templates/alert.jade         |    21 +
 .../frontend/views/templates/batch-confirm.jade |    34 +
 .../frontend/views/templates/clone.jade         |    39 +
 .../frontend/views/templates/confirm.jade       |    33 +
 .../frontend/views/templates/demo-info.jade     |    47 +
 .../frontend/views/templates/dropdown.jade      |    24 +
 .../views/templates/getting-started.jade        |    34 +
 .../frontend/views/templates/message.jade       |    28 +
 .../frontend/views/templates/pagination.jade    |    32 +
 .../frontend/views/templates/select.jade        |    26 +
 .../views/templates/validation-error.jade       |    25 +
 modules/web-console/pom.xml                     |    37 +-
 modules/web-console/src/main/js/.babelrc        |     3 -
 modules/web-console/src/main/js/.eslintrc       |   197 -
 modules/web-console/src/main/js/.gitignore      |    11 -
 .../src/main/js/app/data/colors.json            |    22 -
 .../src/main/js/app/data/countries.json         |    94 -
 .../src/main/js/app/data/demo-info.json         |    14 -
 .../src/main/js/app/data/event-types.json       |   169 -
 .../src/main/js/app/data/getting-started.json   |   109 -
 .../src/main/js/app/data/java-classes.json      |    18 -
 .../src/main/js/app/data/java-keywords.json     |    55 -
 .../src/main/js/app/data/java-primitives.json   |     9 -
 .../src/main/js/app/decorator/select.js         |    77 -
 .../src/main/js/app/decorator/tooltip.js        |    56 -
 .../app/directives/bs-affix-update.directive.js |    34 -
 .../js/app/directives/centered/centered.css     |    37 -
 .../directives/centered/centered.directive.js   |    26 -
 .../hide-on-state-change.directive.js           |    31 -
 .../information/information.directive.js        |    30 -
 .../app/directives/information/information.jade |    20 -
 .../app/directives/information/information.scss |    56 -
 .../ui-ace-docker/ui-ace-docker.controller.js   |    33 -
 .../ui-ace-docker/ui-ace-docker.directive.js    |    46 -
 .../directives/ui-ace-docker/ui-ace-docker.jade |    31 -
 .../ui-ace-java/ui-ace-java.controller.js       |    32 -
 .../ui-ace-java/ui-ace-java.directive.js        |   133 -
 .../app/directives/ui-ace-java/ui-ace-java.jade |    22 -
 .../ui-ace-pojos/ui-ace-pojos.controller.js     |    95 -
 .../ui-ace-pojos/ui-ace-pojos.directive.js      |    46 -
 .../directives/ui-ace-pojos/ui-ace-pojos.jade   |    40 -
 .../ui-ace-pom/ui-ace-pom.controller.js         |    33 -
 .../ui-ace-pom/ui-ace-pom.directive.js          |    41 -
 .../app/directives/ui-ace-pom/ui-ace-pom.jade   |    17 -
 .../js/app/directives/ui-ace-tabs.directive.js  |    23 -
 .../ui-ace-xml/ui-ace-xml.controller.js         |    27 -
 .../ui-ace-xml/ui-ace-xml.directive.js          |   133 -
 .../app/directives/ui-ace-xml/ui-ace-xml.jade   |    17 -
 .../src/main/js/app/filters/byName.filter.js    |    23 -
 .../src/main/js/app/filters/hasPojo.filter.js   |    18 -
 .../src/main/js/app/helpers/jade/mixins.jade    |   587 -
 modules/web-console/src/main/js/app/index.js    |   241 -
 .../src/main/js/app/modules/Demo/Demo.module.js |   165 -
 .../QueryNotebooks/QueryNotebooks.provider.js   |   115 -
 .../js/app/modules/Version/Version.provider.js  |    32 -
 .../src/main/js/app/modules/ace.module.js       |   269 -
 .../js/app/modules/branding/branding.module.js  |    46 -
 .../app/modules/branding/branding.provider.js   |   111 -
 .../app/modules/branding/features.directive.js  |    35 -
 .../js/app/modules/branding/footer.directive.js |    34 -
 .../modules/branding/header-logo.directive.js   |    34 -
 .../js/app/modules/branding/header-logo.jade    |    18 -
 .../modules/branding/header-title.directive.js  |    35 -
 .../branding/powered-by-apache.directive.js     |    35 -
 .../app/modules/branding/powered-by-apache.jade |    18 -
 .../js/app/modules/branding/terms.directive.js  |    30 -
 .../configuration/EventGroups.provider.js       |    30 -
 .../modules/configuration/Sidebar.provider.js   |    39 -
 .../configuration/configuration.module.js       |    41 -
 .../configuration/generator/Docker.service.js   |    78 -
 .../configuration/generator/Java.service.js     |    21 -
 .../configuration/generator/Pom.service.js      |   214 -
 .../configuration/generator/Xml.service.js      |    21 -
 .../modules/configuration/sidebar.directive.js  |    30 -
 .../modules/dialog/dialog-content.directive.js  |    31 -
 .../modules/dialog/dialog-title.directive.js    |    31 -
 .../js/app/modules/dialog/dialog.controller.js  |    40 -
 .../js/app/modules/dialog/dialog.directive.js   |    32 -
 .../js/app/modules/dialog/dialog.factory.js     |    32 -
 .../src/main/js/app/modules/dialog/dialog.jade  |    26 -
 .../main/js/app/modules/dialog/dialog.module.js |    32 -
 .../field/bs-select-placeholder.directive.js    |    46 -
 .../js/app/modules/form/field/down.directive.js |    43 -
 .../modules/form/field/dropdown.directive.js    |    84 -
 .../js/app/modules/form/field/dropdown.jade     |    61 -
 .../main/js/app/modules/form/field/field.css    |    23 -
 .../app/modules/form/field/field.directive.js   |    44 -
 .../main/js/app/modules/form/field/field.jade   |    27 -
 .../field/form-control-feedback.directive.js    |    40 -
 .../form/field/input/autofocus.directive.js     |    30 -
 .../form/field/input/checkbox.directive.js      |    67 -
 .../app/modules/form/field/input/checkbox.jade  |    30 -
 .../form/field/input/datalist.directive.js      |   123 -
 .../app/modules/form/field/input/datalist.jade  |    51 -
 .../form/field/input/number.directive.js        |    77 -
 .../js/app/modules/form/field/input/number.jade |    50 -
 .../js/app/modules/form/field/input/text.css    |    41 -
 .../modules/form/field/input/text.directive.js  |   124 -
 .../js/app/modules/form/field/input/text.jade   |    48 -
 .../app/modules/form/field/label.directive.js   |    47 -
 .../app/modules/form/field/tooltip.directive.js |    49 -
 .../js/app/modules/form/field/up.directive.js   |    44 -
 .../src/main/js/app/modules/form/form.module.js |   101 -
 .../js/app/modules/form/group/add.directive.js  |    40 -
 .../app/modules/form/group/group.directive.js   |    81 -
 .../main/js/app/modules/form/group/group.jade   |    21 -
 .../app/modules/form/group/table.directive.js   |    29 -
 .../main/js/app/modules/form/group/table.jade   |    17 -
 .../app/modules/form/group/tooltip.directive.js |    40 -
 .../app/modules/form/panel/chevron.directive.js |    53 -
 .../app/modules/form/panel/panel.directive.js   |    37 -
 .../app/modules/form/panel/revert.directive.js  |    53 -
 .../form/validator/ipaddress.directive.js       |    86 -
 .../validator/java-built-in-class.directive.js  |    31 -
 .../form/validator/java-identifier.directive.js |    31 -
 .../form/validator/java-keywords.directive.js   |    43 -
 .../validator/java-package-name.directive.js    |    31 -
 .../java-package-specified.directive.js         |    34 -
 .../form/validator/property-unique.directive.js |    47 -
 .../property-value-specified.directive.js       |    31 -
 .../modules/form/validator/unique.directive.js  |    49 -
 .../getting-started/GettingStarted.provider.js  |   112 -
 .../src/main/js/app/modules/loading/loading.css |    73 -
 .../js/app/modules/loading/loading.directive.js |    51 -
 .../main/js/app/modules/loading/loading.jade    |    23 -
 .../js/app/modules/loading/loading.module.js    |    26 -
 .../js/app/modules/loading/loading.service.js   |    48 -
 .../js/app/modules/navbar/Navbar.provider.js    |    28 -
 .../js/app/modules/navbar/Userbar.provider.js   |    28 -
 .../js/app/modules/navbar/navbar.directive.js   |    30 -
 .../main/js/app/modules/navbar/navbar.module.js |    33 -
 .../js/app/modules/navbar/userbar.directive.js  |    48 -
 .../src/main/js/app/modules/socket.module.js    |    41 -
 .../main/js/app/modules/states/admin.state.js   |    34 -
 .../app/modules/states/configuration.state.js   |   202 -
 .../caches/concurrency.directive.js             |    27 -
 .../configuration/caches/concurrency.jade       |    65 -
 .../configuration/caches/general.directive.js   |    27 -
 .../states/configuration/caches/general.jade    |    65 -
 .../configuration/caches/memory.directive.js    |    27 -
 .../states/configuration/caches/memory.jade     |    88 -
 .../configuration/caches/query.directive.js     |    27 -
 .../states/configuration/caches/query.jade      |    93 -
 .../configuration/caches/rebalance.directive.js |    27 -
 .../states/configuration/caches/rebalance.jade  |    65 -
 .../caches/server-near-cache.directive.js       |    27 -
 .../configuration/caches/server-near-cache.jade |    45 -
 .../caches/statistics.directive.js              |    27 -
 .../states/configuration/caches/statistics.jade |    37 -
 .../configuration/caches/store.directive.js     |    27 -
 .../states/configuration/caches/store.jade      |   266 -
 .../configuration/clusters/atomic.directive.js  |    27 -
 .../states/configuration/clusters/atomic.jade   |    53 -
 .../configuration/clusters/binary.directive.js  |    27 -
 .../states/configuration/clusters/binary.jade   |   100 -
 .../clusters/communication.directive.js         |    27 -
 .../configuration/clusters/communication.jade   |    90 -
 .../clusters/connector.directive.js             |    27 -
 .../configuration/clusters/connector.jade       |   103 -
 .../clusters/deployment.directive.js            |    27 -
 .../configuration/clusters/deployment.jade      |   112 -
 .../clusters/discovery.directive.js             |    27 -
 .../configuration/clusters/discovery.jade       |    81 -
 .../configuration/clusters/events.directive.js  |    27 -
 .../states/configuration/clusters/events.jade   |    37 -
 .../configuration/clusters/general.directive.js |    27 -
 .../states/configuration/clusters/general.jade  |    70 -
 .../general/discovery/cloud.directive.js        |    27 -
 .../clusters/general/discovery/cloud.jade       |   125 -
 .../general/discovery/google.directive.js       |    27 -
 .../clusters/general/discovery/google.jade      |    35 -
 .../general/discovery/jdbc.directive.js         |    27 -
 .../clusters/general/discovery/jdbc.jade        |    24 -
 .../general/discovery/multicast.directive.js    |    27 -
 .../clusters/general/discovery/multicast.jade   |   109 -
 .../clusters/general/discovery/s3.directive.js  |    27 -
 .../clusters/general/discovery/s3.jade          |    25 -
 .../general/discovery/shared.directive.js       |    27 -
 .../clusters/general/discovery/shared.jade      |    23 -
 .../clusters/general/discovery/vm.directive.js  |    27 -
 .../clusters/general/discovery/vm.jade          |    90 -
 .../general/discovery/zookeeper.directive.js    |    27 -
 .../clusters/general/discovery/zookeeper.jade   |    73 -
 .../bounded-exponential-backoff.directive.js    |    27 -
 .../bounded-exponential-backoff.jade            |    27 -
 .../zookeeper/retrypolicy/custom.directive.js   |    27 -
 .../discovery/zookeeper/retrypolicy/custom.jade |    22 -
 .../exponential-backoff.directive.js            |    27 -
 .../retrypolicy/exponential-backoff.jade        |    27 -
 .../zookeeper/retrypolicy/forever.directive.js  |    27 -
 .../zookeeper/retrypolicy/forever.jade          |    22 -
 .../zookeeper/retrypolicy/n-times.directive.js  |    27 -
 .../zookeeper/retrypolicy/n-times.jade          |    25 -
 .../zookeeper/retrypolicy/one-time.directive.js |    27 -
 .../zookeeper/retrypolicy/one-time.jade         |    23 -
 .../retrypolicy/until-elapsed.directive.js      |    27 -
 .../zookeeper/retrypolicy/until-elapsed.jade    |    25 -
 .../configuration/clusters/igfs.directive.js    |    27 -
 .../states/configuration/clusters/igfs.jade     |    37 -
 .../clusters/marshaller.directive.js            |    27 -
 .../configuration/clusters/marshaller.jade      |    69 -
 .../configuration/clusters/metrics.directive.js |    27 -
 .../states/configuration/clusters/metrics.jade  |    50 -
 .../configuration/clusters/ssl.directive.js     |    27 -
 .../states/configuration/clusters/ssl.jade      |   108 -
 .../configuration/clusters/swap.directive.js    |    27 -
 .../states/configuration/clusters/swap.jade     |    67 -
 .../configuration/clusters/thread.directive.js  |    27 -
 .../states/configuration/clusters/thread.jade   |    48 -
 .../configuration/clusters/time.directive.js    |    27 -
 .../states/configuration/clusters/time.jade     |    47 -
 .../clusters/transactions.directive.js          |    27 -
 .../configuration/clusters/transactions.jade    |    59 -
 .../configuration/domains/general.directive.js  |    27 -
 .../states/configuration/domains/general.jade   |    46 -
 .../configuration/domains/query.directive.js    |    27 -
 .../states/configuration/domains/query.jade     |   169 -
 .../configuration/domains/store.directive.js    |    27 -
 .../states/configuration/domains/store.jade     |   126 -
 .../states/configuration/igfs/dual.directive.js |    27 -
 .../modules/states/configuration/igfs/dual.jade |    42 -
 .../igfs/fragmentizer.directive.js              |    27 -
 .../states/configuration/igfs/fragmentizer.jade |    43 -
 .../configuration/igfs/general.directive.js     |    27 -
 .../states/configuration/igfs/general.jade      |    53 -
 .../states/configuration/igfs/ipc.directive.js  |    27 -
 .../modules/states/configuration/igfs/ipc.jade  |    57 -
 .../states/configuration/igfs/misc.directive.js |    27 -
 .../modules/states/configuration/igfs/misc.jade |   108 -
 .../configuration/igfs/secondary.directive.js   |    27 -
 .../states/configuration/igfs/secondary.jade    |    44 -
 .../configuration/preview-panel.directive.js    |   239 -
 .../summary/summary-tabs.directive.js           |    50 -
 .../configuration/summary/summary.controller.js |   360 -
 .../configuration/summary/summary.resource.js   |    40 -
 .../main/js/app/modules/states/logout.state.js  |    36 -
 .../js/app/modules/states/password.state.js     |    46 -
 .../main/js/app/modules/states/profile.state.js |    34 -
 .../main/js/app/modules/states/signin.state.js  |    52 -
 .../src/main/js/app/modules/states/sql.state.js |    46 -
 .../main/js/app/modules/user/Auth.service.js    |    73 -
 .../main/js/app/modules/user/User.service.js    |    65 -
 .../src/main/js/app/modules/user/user.module.js |    28 -
 .../js/app/services/AgentMonitor.service.js     |   337 -
 .../main/js/app/services/ChartColors.service.js |    22 -
 .../main/js/app/services/Countries.service.js   |    31 -
 .../main/js/app/services/InetAddress.service.js |    53 -
 .../main/js/app/services/JavaTypes.service.js   |    84 -
 .../src/main/js/app/services/cleanup.service.js |    44 -
 .../src/main/js/app/services/confirm.service.js |    70 -
 .../src/main/js/build/system.config.js          |   415 -
 .../src/main/js/controllers/admin-controller.js |    92 -
 .../main/js/controllers/caches-controller.js    |   493 -
 .../main/js/controllers/clusters-controller.js  |   555 -
 .../src/main/js/controllers/common-module.js    |  1759 ---
 .../main/js/controllers/domains-controller.js   |  1786 ---
 .../src/main/js/controllers/igfs-controller.js  |   441 -
 .../main/js/controllers/profile-controller.js   |    84 -
 .../src/main/js/controllers/sql-controller.js   |  1568 --
 .../src/main/js/generator/generator-common.js   |   570 -
 .../src/main/js/generator/generator-java.js     |  3179 ----
 .../src/main/js/generator/generator-optional.js |    25 -
 .../main/js/generator/generator-properties.js   |   149 -
 .../src/main/js/generator/generator-readme.js   |    85 -
 .../src/main/js/generator/generator-xml.js      |  1769 ---
 .../src/main/js/gulpfile.babel.js/index.js      |    26 -
 .../src/main/js/gulpfile.babel.js/paths.js      |   101 -
 .../main/js/gulpfile.babel.js/tasks/build.js    |    21 -
 .../main/js/gulpfile.babel.js/tasks/bundle.js   |    76 -
 .../main/js/gulpfile.babel.js/tasks/clean.js    |    35 -
 .../main/js/gulpfile.babel.js/tasks/connect.js  |    47 -
 .../src/main/js/gulpfile.babel.js/tasks/copy.js |    57 -
 .../main/js/gulpfile.babel.js/tasks/eslint.js   |    43 -
 .../gulpfile.babel.js/tasks/ignite-modules.js   |    56 -
 .../src/main/js/gulpfile.babel.js/tasks/jade.js |    40 -
 .../src/main/js/gulpfile.babel.js/tasks/sass.js |    25 -
 .../main/js/gulpfile.babel.js/tasks/watch.js    |    39 -
 .../src/main/js/ignite_modules/README.txt       |     6 -
 .../src/main/js/ignite_modules/index.js         |    27 -
 modules/web-console/src/main/js/package.json    |   271 -
 .../web-console/src/main/js/public/favicon.ico  |   Bin 1150 -> 0 bytes
 .../src/main/js/public/images/cache.png         |   Bin 23700 -> 0 bytes
 .../src/main/js/public/images/cluster.png       |   Bin 29670 -> 0 bytes
 .../src/main/js/public/images/docker.png        |   Bin 521 -> 0 bytes
 .../src/main/js/public/images/domains.png       |   Bin 23828 -> 0 bytes
 .../src/main/js/public/images/igfs.png          |   Bin 14307 -> 0 bytes
 .../src/main/js/public/images/ignite-logo.png   |   Bin 1982 -> 0 bytes
 .../main/js/public/images/ignite-logo@2x.png    |   Bin 3325 -> 0 bytes
 .../src/main/js/public/images/ignite-puzzle.png |   Bin 71974 -> 0 bytes
 .../src/main/js/public/images/java.png          |   Bin 170 -> 0 bytes
 .../src/main/js/public/images/pb-ignite.png     |   Bin 3493 -> 0 bytes
 .../src/main/js/public/images/pb-ignite@2x.png  |   Bin 8558 -> 0 bytes
 .../src/main/js/public/images/query-chart.png   |   Bin 16637 -> 0 bytes
 .../main/js/public/images/query-metadata.png    |   Bin 32298 -> 0 bytes
 .../src/main/js/public/images/query-table.png   |   Bin 42253 -> 0 bytes
 .../src/main/js/public/images/summary.png       |   Bin 31997 -> 0 bytes
 .../src/main/js/public/images/xml.png           |   Bin 232 -> 0 bytes
 .../public/stylesheets/_bootstrap-custom.scss   |    65 -
 .../stylesheets/_bootstrap-variables.scss       |   891 --
 .../stylesheets/_font-awesome-custom.scss       |    31 -
 .../src/main/js/public/stylesheets/style.scss   |  2128 ---
 .../main/js/public/stylesheets/variables.scss   |    28 -
 modules/web-console/src/main/js/serve.js        |   116 -
 modules/web-console/src/main/js/serve/agent.js  |   601 -
 .../src/main/js/serve/agent_dists/README.txt    |     7 -
 modules/web-console/src/main/js/serve/app.js    |    42 -
 .../web-console/src/main/js/serve/browser.js    |   304 -
 .../main/js/serve/config/settings.json.sample   |    26 -
 .../web-console/src/main/js/serve/configure.js  |    83 -
 .../web-console/src/main/js/serve/keys/test.crt |    13 -
 .../web-console/src/main/js/serve/keys/test.key |    18 -
 modules/web-console/src/main/js/serve/mail.js   |    75 -
 modules/web-console/src/main/js/serve/mongo.js  |   620 -
 .../src/main/js/serve/routes/admin.js           |   126 -
 .../src/main/js/serve/routes/agent.js           |    82 -
 .../src/main/js/serve/routes/caches.js          |   132 -
 .../src/main/js/serve/routes/clusters.js        |   146 -
 .../src/main/js/serve/routes/demo.js            |   135 -
 .../src/main/js/serve/routes/demo/caches.json   |    87 -
 .../src/main/js/serve/routes/demo/clusters.json |    50 -
 .../src/main/js/serve/routes/demo/domains.json  |   307 -
 .../src/main/js/serve/routes/demo/igfss.json    |    10 -
 .../src/main/js/serve/routes/domains.js         |   195 -
 .../src/main/js/serve/routes/igfs.js            |   122 -
 .../src/main/js/serve/routes/notebooks.js       |   121 -
 .../src/main/js/serve/routes/profile.js         |    95 -
 .../src/main/js/serve/routes/public.js          |   228 -
 .../src/main/js/serve/routes/routes.js          |   103 -
 .../web-console/src/main/js/serve/settings.js   |    84 -
 modules/web-console/src/main/js/views/base.jade |    22 -
 .../src/main/js/views/configuration/caches.jade |    52 -
 .../main/js/views/configuration/clusters.jade   |    60 -
 .../js/views/configuration/domains-import.jade  |   196 -
 .../main/js/views/configuration/domains.jade    |    66 -
 .../src/main/js/views/configuration/igfs.jade   |    51 -
 .../main/js/views/configuration/sidebar.jade    |    29 -
 .../summary-project-structure.jade              |    27 -
 .../js/views/configuration/summary-tabs.jade    |    25 -
 .../main/js/views/configuration/summary.jade    |   152 -
 .../src/main/js/views/includes/footer.jade      |    23 -
 .../src/main/js/views/includes/header.jade      |    48 -
 .../web-console/src/main/js/views/index.jade    |    66 -
 .../web-console/src/main/js/views/reset.jade    |    48 -
 .../src/main/js/views/settings/admin.jade       |    76 -
 .../src/main/js/views/settings/profile.jade     |    76 -
 .../web-console/src/main/js/views/signin.jade   |   159 -
 .../src/main/js/views/sql/cache-metadata.jade   |    40 -
 .../src/main/js/views/sql/chart-settings.jade   |    40 -
 .../src/main/js/views/sql/notebook-new.jade     |    31 -
 .../src/main/js/views/sql/paragraph-rate.jade   |    31 -
 .../web-console/src/main/js/views/sql/sql.jade  |   180 -
 .../main/js/views/templates/agent-download.jade |    48 -
 .../src/main/js/views/templates/alert.jade      |    21 -
 .../main/js/views/templates/batch-confirm.jade  |    32 -
 .../src/main/js/views/templates/clone.jade      |    31 -
 .../src/main/js/views/templates/confirm.jade    |    31 -
 .../src/main/js/views/templates/demo-info.jade  |    45 -
 .../src/main/js/views/templates/dropdown.jade   |    21 -
 .../js/views/templates/getting-started.jade     |    32 -
 .../src/main/js/views/templates/message.jade    |    26 -
 .../src/main/js/views/templates/pagination.jade |    32 -
 .../src/main/js/views/templates/select.jade     |    26 -
 .../js/views/templates/validation-error.jade    |    25 -
 modules/web-console/src/test/js/routes/agent.js |    94 -
 modules/web-console/web-agent/.gitignore        |     2 +
 modules/web-console/web-agent/README.txt        |    88 +
 .../web-agent/assembly/release-web-agent.xml    |    66 +
 .../web-agent/bin/ignite-web-agent.bat          |    70 +
 .../web-agent/bin/ignite-web-agent.sh           |    87 +
 modules/web-console/web-agent/demo/README.txt   |     4 +
 modules/web-console/web-agent/demo/db-init.sql  |   102 +
 .../web-agent/jdbc-drivers/README.txt           |    10 +
 modules/web-console/web-agent/logs/README.txt   |     5 +
 modules/web-console/web-agent/pom.xml           |   199 +
 .../console/agent/AgentConfiguration.java       |   268 +
 .../ignite/console/agent/AgentLauncher.java     |   344 +
 .../apache/ignite/console/agent/AgentUtils.java |   111 +
 .../console/agent/handlers/AbstractHandler.java |   110 +
 .../console/agent/handlers/DatabaseHandler.java |   298 +
 .../console/agent/handlers/RestHandler.java     |   276 +
 .../ignite/console/demo/AgentClusterDemo.java   |   641 +
 .../ignite/console/demo/AgentMetadataDemo.java  |    92 +
 .../apache/ignite/console/demo/model/Car.java   |   152 +
 .../ignite/console/demo/model/Country.java      |   152 +
 .../ignite/console/demo/model/Department.java   |   152 +
 .../ignite/console/demo/model/Employee.java     |   356 +
 .../ignite/console/demo/model/Parking.java      |   152 +
 .../src/main/resources/log4j.properties         |    53 +
 modules/web/ignite-appserver-test/pom.xml       |     5 +-
 modules/web/ignite-websphere-test/pom.xml       |     5 +-
 modules/web/pom.xml                             |     2 +-
 .../cache/websession/WebSessionFilter.java      |    46 +-
 .../ignite/cache/websession/WebSessionV2.java   |    22 +-
 .../IgniteWebSessionSelfTestSuite.java          |    68 +-
 .../WebSessionReplicatedSelfTest.java           |    28 +
 .../WebSessionReplicatedV1SelfTest.java         |    28 +
 .../internal/websession/WebSessionSelfTest.java |   332 +-
 .../WebSessionTransactionalSelfTest.java        |    48 +
 .../WebSessionTransactionalV1SelfTest.java      |    28 +
 .../websession/WebSessionV1SelfTest.java        |    28 +
 .../config/benchmark-client-mode.properties     |     2 +
 .../yardstick/config/benchmark-full.properties  |   116 +
 .../config/benchmark-tx-win.properties          |     2 +
 .../yardstick/config/benchmark-tx.properties    |     2 +
 .../yardstick/config/benchmark-win.properties   |     2 +
 modules/yardstick/config/benchmark.properties   |     2 +
 .../config/ignite-base-load-config.xml          |     4 +-
 modules/yardstick/config/queries.sql            |     1 +
 modules/yardstick/pom.xml                       |    48 +-
 .../ignite/yardstick/IgniteBenchmarkUtils.java  |    71 +-
 .../org/apache/ignite/yardstick/IgniteNode.java |    74 +-
 .../cache/IgniteAtomicSequenceBenchmark.java    |    47 +
 .../cache/IgniteGetAndPutBenchmark.java         |    41 +
 .../cache/IgniteGetAndPutTxBenchmark.java       |    70 +
 .../cache/IgniteInvokeTxBenchmark.java          |    40 +
 .../IgniteInvokeWithInjectionBenchmark.java     |    74 +
 .../IgniteInvokeWithInjectionTxBenchmark.java   |    30 +
 .../IgniteSqlQueryDistributedJoinBenchmark.java |   166 +
 ...lQueryDistributedJoinBroadcastBenchmark.java |    28 +
 .../IgniteCacheRandomOperationBenchmark.java    |   751 +-
 .../yardstick/cache/load/model/ModelUtil.java   |    44 +-
 .../cache/load/model/key/Identifier.java        |     6 +
 .../yardstick/cache/load/model/key/Mark.java    |     7 +
 .../yardstick/cache/load/model/value/Car.java   |     6 +
 .../yardstick/cache/load/model/value/Truck.java |    12 +-
 .../ignite/yardstick/cache/model/Account.java   |     6 +
 .../ignite/yardstick/cache/model/Person1.java   |     2 +-
 .../ignite/yardstick/cache/model/Person2.java   |     2 +-
 .../ignite/yardstick/cache/model/SampleKey.java |     2 +-
 .../yardstick/cache/model/SampleValue.java      |     2 +-
 modules/yarn/pom.xml                            |     2 +-
 modules/zookeeper/pom.xml                       |     5 +-
 .../zk/TcpDiscoveryZookeeperIpFinder.java       |    84 +-
 .../tcp/ipfinder/zk/ZookeeperIpFinderTest.java  |    16 +-
 parent/pom.xml                                  |    50 +-
 pom.xml                                         |    11 +-
 3328 files changed, 286582 insertions(+), 161927 deletions(-)
----------------------------------------------------------------------



[17/50] [abbrv] ignite git commit: ignite-2971 Enabled tests

Posted by vo...@apache.org.
ignite-2971 Enabled tests


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

Branch: refs/heads/ignite-2693
Commit: 45bb1ac95a9385f9a03388427cd1df25e9993c42
Parents: 138a3aa
Author: sboikov <sb...@gridgain.com>
Authored: Fri Nov 4 16:36:05 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Nov 4 16:36:05 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/WithKeepBinaryCacheFullApiTest.java  | 10 +---------
 1 file changed, 1 insertion(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/45bb1ac9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WithKeepBinaryCacheFullApiTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WithKeepBinaryCacheFullApiTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WithKeepBinaryCacheFullApiTest.java
index 1954a8d..3e6b0b0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WithKeepBinaryCacheFullApiTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/WithKeepBinaryCacheFullApiTest.java
@@ -459,11 +459,7 @@ public class WithKeepBinaryCacheFullApiTest extends IgniteCacheConfigVariationsA
 
         for (TransactionConcurrency conc : TransactionConcurrency.values()) {
             for (TransactionIsolation isolation : TransactionIsolation.values()) {
-                // TODO IGNITE-2971: delete this if when the issue will be fixed.
-                if (conc == TransactionConcurrency.OPTIMISTIC && isolation == TransactionIsolation.SERIALIZABLE)
-                    continue;
-
-                info(">>>>> Executing test using explicite txs [concurrency=" + conc + ", isolation=" + isolation + "]");
+                info(">>>>> Executing test using explicit txs [concurrency=" + conc + ", isolation=" + isolation + "]");
 
                 checkInvokeTx(conc, isolation);
 
@@ -671,10 +667,6 @@ public class WithKeepBinaryCacheFullApiTest extends IgniteCacheConfigVariationsA
 
         for (TransactionConcurrency conc : TransactionConcurrency.values()) {
             for (TransactionIsolation isolation : TransactionIsolation.values()) {
-                // TODO IGNITE-2971: delete this if when the issue will be fixed.
-                if (conc == TransactionConcurrency.OPTIMISTIC && isolation == TransactionIsolation.SERIALIZABLE)
-                    continue;
-
                 checkInvokeAsyncTx(conc, isolation);
 
                 jcache().removeAll();


[36/50] [abbrv] ignite git commit: .NET: Fix minor analysis warnings

Posted by vo...@apache.org.
.NET: Fix minor analysis warnings


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

Branch: refs/heads/ignite-2693
Commit: 986e2769b3e1112a601e1688231a500afe23360a
Parents: 5b31d83
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Nov 8 14:07:54 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Wed Nov 9 19:53:36 2016 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs   | 2 +-
 .../dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs    | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/986e2769/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs
index f6a2f07..86b1406 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Compute/ComputeTaskSplitAdapter.cs
@@ -62,7 +62,7 @@ namespace Apache.Ignite.Core.Compute
         /// </returns>
         /// <exception cref="IgniteException">Split returned no jobs.</exception>
         [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
-        override public IDictionary<IComputeJob<TJobRes>, IClusterNode> Map(IList<IClusterNode> subgrid, TArg arg)
+        public override IDictionary<IComputeJob<TJobRes>, IClusterNode> Map(IList<IClusterNode> subgrid, TArg arg)
         {
             var jobs = Split(subgrid.Count, arg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/986e2769/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs
index e54a199..a5e2cb0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedTarget.cs
@@ -24,7 +24,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
     /// <summary>
     /// Base unmanaged target implementation.
     /// </summary>
-    internal unsafe sealed class UnmanagedTarget : CriticalHandle, IUnmanagedTarget
+    internal sealed unsafe class UnmanagedTarget : CriticalHandle, IUnmanagedTarget
     {
         /** Context. */
         private readonly UnmanagedContext _ctx;


[28/50] [abbrv] ignite git commit: Merge branch 'ignite-1.7.3'

Posted by vo...@apache.org.
Merge branch 'ignite-1.7.3'

# Conflicts:
#	modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheGetEntryAbstractTest.java
#	modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
#	modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs
#	modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
#	modules/platforms/dotnet/Apache.Ignite.NLog/packages.config
#	modules/platforms/dotnet/Apache.Ignite.sln.DotSettings


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

Branch: refs/heads/ignite-2693
Commit: 173e2b83d2059e7f70e357387b860f4d0c9cd14f
Parents: 3d9f892 b1c7c9b
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Nov 9 11:13:27 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Nov 9 11:13:27 2016 +0300

----------------------------------------------------------------------
 .../internal/jdbc2/JdbcResultSetSelfTest.java   |   91 +-
 .../java/org/apache/ignite/IgniteScheduler.java |   13 +
 .../rendezvous/RendezvousAffinityFunction.java  |    2 +-
 .../cache/query/QueryCancelledException.java    |   35 +
 .../apache/ignite/cache/query/QueryCursor.java  |    8 +-
 .../org/apache/ignite/cache/query/SpiQuery.java |    8 +-
 .../ignite/cache/query/SqlFieldsQuery.java      |   26 +
 .../org/apache/ignite/cache/query/SqlQuery.java |   25 +
 .../cache/store/jdbc/CacheJdbcBlobStore.java    |    4 +-
 .../configuration/CacheConfiguration.java       |   13 +-
 .../configuration/IgniteConfiguration.java      |   18 +-
 .../local/LocalIgfsSecondaryFileSystem.java     |   90 +-
 .../internal/GridEventConsumeHandler.java       |    8 +-
 .../internal/GridJobSiblingsResponse.java       |    3 +-
 .../ignite/internal/GridKernalContext.java      |   23 +
 .../ignite/internal/GridKernalContextImpl.java  |   40 +-
 .../internal/GridMessageListenHandler.java      |    8 +-
 .../apache/ignite/internal/IgniteKernal.java    |   13 +-
 .../ignite/internal/IgniteSchedulerImpl.java    |   18 +
 .../org/apache/ignite/internal/IgnitionEx.java  |  125 +-
 .../ignite/internal/MarshallerContextImpl.java  |    5 +-
 .../internal/binary/BinaryEnumObjectImpl.java   |    3 +-
 .../internal/binary/BinaryMarshaller.java       |    2 +-
 .../internal/binary/BinaryWriterExImpl.java     |    3 +-
 .../GridClientOptimizedMarshaller.java          |    5 +-
 .../GridClientZipOptimizedMarshaller.java       |    5 +-
 .../apache/ignite/internal/jdbc/JdbcUtils.java  |    4 +-
 .../checkpoint/GridCheckpointManager.java       |    6 +-
 .../managers/communication/GridIoManager.java   |  190 +-
 .../deployment/GridDeploymentCommunication.java |    4 +-
 .../eventstorage/GridEventStorageManager.java   |   16 +-
 .../swapspace/GridSwapSpaceManager.java         |   13 +-
 .../processors/affinity/GridAffinityUtils.java  |    4 +-
 .../cache/CacheEntrySerializablePredicate.java  |    4 +-
 .../cache/CacheInvokeDirectResult.java          |    4 +-
 .../processors/cache/GridCacheMapEntry.java     |   13 +-
 .../processors/cache/GridCacheMessage.java      |    4 +-
 .../processors/cache/GridCacheProcessor.java    |   13 +-
 .../processors/cache/GridCacheUtils.java        |    2 +-
 .../processors/cache/IgniteCacheProxy.java      |   34 +-
 .../processors/cache/QueryCursorImpl.java       |   92 +-
 .../GridDistributedLockResponse.java            |    4 +-
 .../GridDistributedTxPrepareRequest.java        |    4 +-
 .../GridDistributedTxPrepareResponse.java       |    4 +-
 .../dht/GridDhtAffinityAssignmentRequest.java   |    2 +-
 .../dht/GridDhtAffinityAssignmentResponse.java  |   12 +-
 .../dht/GridDhtTxFinishResponse.java            |    4 +-
 .../dht/atomic/GridDhtAtomicUpdateResponse.java |    4 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |    2 +-
 .../atomic/GridNearAtomicUpdateResponse.java    |    4 +-
 .../colocated/GridDhtDetachedCacheEntry.java    |    5 +
 .../dht/preloader/GridDhtForceKeysResponse.java |    4 +-
 .../GridDhtPartitionDemandMessage.java          |    4 +-
 .../preloader/GridDhtPartitionsFullMessage.java |    8 +-
 .../GridDhtPartitionsSingleMessage.java         |    8 +-
 .../distributed/near/GridNearGetResponse.java   |    4 +-
 .../near/GridNearSingleGetResponse.java         |    4 +-
 .../near/GridNearTxFinishResponse.java          |    4 +-
 .../cache/distributed/near/GridNearTxLocal.java |    2 +-
 .../cache/query/GridCacheLocalQueryFuture.java  |    4 +-
 .../cache/query/GridCacheQueryManager.java      |    2 +-
 .../cache/query/GridCacheQueryRequest.java      |   12 +-
 .../cache/query/GridCacheQueryResponse.java     |    4 +-
 .../cache/query/GridCacheSqlQuery.java          |    4 +-
 .../continuous/CacheContinuousQueryHandler.java |   12 +-
 .../jdbc/GridCacheQueryJdbcMetadataTask.java    |    4 +-
 .../query/jdbc/GridCacheQueryJdbcTask.java      |    6 +-
 .../cache/transactions/IgniteTxEntry.java       |    5 +-
 .../transactions/IgniteTxLocalAdapter.java      |   62 +-
 .../version/GridCacheRawVersionedEntry.java     |    8 +-
 .../IgniteCacheObjectProcessorImpl.java         |    3 +-
 .../closure/GridClosureProcessor.java           |   77 +-
 .../processors/cluster/ClusterProcessor.java    |    2 +-
 .../processors/cluster/GridUpdateNotifier.java  |  108 +-
 .../continuous/GridContinuousProcessor.java     |   12 +-
 .../processors/continuous/StartRequestData.java |    4 +-
 .../datastreamer/DataStreamProcessor.java       |    8 +-
 .../datastreamer/DataStreamerImpl.java          |    6 +-
 .../processors/hadoop/HadoopHelper.java         |    7 +
 .../internal/processors/hadoop/HadoopJob.java   |    5 +
 .../hadoop/HadoopMapReducePlanner.java          |    1 +
 .../processors/hadoop/HadoopNoopHelper.java     |    5 +
 .../processors/igfs/IgfsAckMessage.java         |    5 +-
 .../processors/igfs/IgfsDeleteMessage.java      |    5 +-
 .../internal/processors/igfs/IgfsImpl.java      |   11 +-
 .../processors/igfs/IgfsMetaManager.java        |    1 -
 .../internal/processors/igfs/IgfsPaths.java     |    4 +-
 .../internal/processors/igfs/IgfsServer.java    |    5 +-
 .../processors/igfs/IgfsThreadFactory.java      |   61 +
 .../local/LocalFileSystemIgfsFile.java          |    9 +-
 .../secondary/local/LocalFileSystemUtils.java   |  142 +
 .../processors/job/GridJobProcessor.java        |   18 +-
 .../internal/processors/job/GridJobWorker.java  |   16 +-
 .../offheap/GridOffHeapProcessor.java           |    4 +-
 .../platform/PlatformAbstractTarget.java        |  177 +-
 .../processors/platform/PlatformTarget.java     |   44 +-
 .../platform/cache/PlatformCache.java           |  661 +++-
 .../cache/affinity/PlatformAffinity.java        |   14 +-
 .../query/PlatformAbstractQueryCursor.java      |   42 +-
 .../query/PlatformContinuousQueryProxy.java     |   53 +
 .../cache/store/PlatformCacheStoreCallback.java |   61 -
 .../callback/PlatformCallbackGateway.java       |    5 +-
 .../callback/PlatformCallbackUtils.java         |    3 +-
 .../platform/cluster/PlatformClusterGroup.java  |  124 +-
 .../platform/compute/PlatformCompute.java       |  106 +-
 .../cpp/PlatformCppConfigurationClosure.java    |   13 +-
 .../datastreamer/PlatformDataStreamer.java      |  112 +-
 .../datastructures/PlatformAtomicLong.java      |  179 +-
 .../datastructures/PlatformAtomicReference.java |   38 +-
 .../datastructures/PlatformAtomicSequence.java  |  131 +-
 .../dotnet/PlatformDotNetCacheStore.java        |  117 +-
 .../PlatformDotNetConfigurationClosure.java     |   13 +-
 .../platform/events/PlatformEvents.java         |  194 +-
 .../platform/messaging/PlatformMessaging.java   |   83 +-
 .../platform/services/PlatformServices.java     |  223 +-
 .../transactions/PlatformTransactions.java      |  225 +-
 .../platform/utils/PlatformUtils.java           |    8 +-
 .../internal/processors/pool/PoolProcessor.java |  154 +
 .../processors/query/GridQueryCancel.java       |   83 +
 .../processors/query/GridQueryFieldsResult.java |    3 +-
 .../query/GridQueryFieldsResultAdapter.java     |    3 +-
 .../processors/query/GridQueryIndexing.java     |   11 +-
 .../processors/query/GridQueryProcessor.java    |  176 +-
 .../twostep/messages/GridQueryFailResponse.java |   36 +-
 .../h2/twostep/messages/GridQueryRequest.java   |   31 +-
 .../handlers/task/GridTaskCommandHandler.java   |   10 +-
 .../rest/protocols/tcp/GridTcpRestParser.java   |    4 +-
 .../service/GridServiceProcessor.java           |   70 +-
 .../processors/task/GridTaskProcessor.java      |    8 +-
 .../processors/task/GridTaskWorker.java         |   14 +-
 .../ignite/internal/util/IgniteUtils.java       |  291 +-
 .../internal/util/SerializableTransient.java    |   58 +
 .../shmem/IpcSharedMemoryServerEndpoint.java    |   15 +-
 .../util/nio/GridNioSessionMetaKey.java         |    7 +-
 .../util/nio/ssl/BlockingSslHandler.java        |   61 +-
 .../internal/util/nio/ssl/GridNioSslFilter.java |   63 +-
 .../util/nio/ssl/GridNioSslHandler.java         |   17 +-
 .../internal/util/nio/ssl/GridSslMeta.java      |   94 +
 .../visor/cache/VisorCacheTypeMetadata.java     |   70 +-
 .../apache/ignite/logger/java/JavaLogger.java   |   14 +-
 .../logger/java/JavaLoggerFileHandler.java      |   10 +-
 .../ignite/marshaller/MarshallerUtils.java      |   22 +
 .../ignite/marshaller/jdk/JdkMarshaller.java    |    8 +-
 .../optimized/OptimizedClassDescriptor.java     |   90 +-
 .../optimized/OptimizedMarshaller.java          |   14 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |    3 +-
 .../spi/checkpoint/sharedfs/SharedFsUtils.java  |    4 +-
 .../communication/tcp/TcpCommunicationSpi.java  |   53 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |    8 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   24 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |    8 +-
 .../TcpDiscoveryMulticastIpFinder.java          |    4 +-
 .../sharedfs/TcpDiscoverySharedFsIpFinder.java  |    2 +-
 .../TcpDiscoveryCustomEventMessage.java         |    3 +-
 .../RoundRobinGlobalLoadBalancer.java           |   16 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |    4 +-
 .../ignite/stream/socket/SocketStreamer.java    |    3 +-
 ...RoundRobinErrorAfterClientReconnectTest.java |   99 +
 .../MarshallerContextLockingSelfTest.java       |    2 +-
 .../managers/GridManagerStopSelfTest.java       |    2 +
 .../cache/CacheGetEntryAbstractTest.java        |    3 +-
 .../processors/cache/CachePutIfAbsentTest.java  |  161 +
 .../CacheSerializableTransactionsTest.java      |   70 +-
 .../processors/cache/CacheTxFastFinishTest.java |    2 +-
 .../GridCacheTtlManagerEvictionSelfTest.java    |  160 +
 .../MarshallerCacheJobRunNodeRestartTest.java   |    2 +-
 .../cache/query/IndexingSpiQuerySelfTest.java   |  218 ++
 .../cache/query/IndexingSpiQueryTxSelfTest.java |  162 +
 ...eContinuousQueryAsyncFailoverTxSelfTest.java |    5 +
 ...ContinuousQueryFailoverAbstractSelfTest.java |  225 +-
 .../CacheContinuousQueryFailoverTxSelfTest.java |    5 +
 ...eEntryProcessorExternalizableFailedTest.java |  588 +++
 .../igfs/IgfsAbstractBaseSelfTest.java          |   19 +
 .../processors/igfs/IgfsAbstractSelfTest.java   |  146 +-
 .../igfs/IgfsDualAbstractSelfTest.java          |   42 +-
 ...SecondaryFileSystemDualAbstractSelfTest.java |   26 +-
 ...fsLocalSecondaryFileSystemProxySelfTest.java |    5 -
 ...IgfsLocalSecondaryFileSystemTestAdapter.java |   27 +-
 .../igfs/benchmark/IgfsBenchmark.java           |  561 +++
 .../handlers/log/GridLogCommandHandlerTest.java |    5 +-
 ...artupWithSpecifiedWorkDirectorySelfTest.java |  166 -
 .../util/future/GridFutureAdapterSelfTest.java  |    3 +
 .../ipc/shmem/IgfsSharedMemoryTestServer.java   |    4 +-
 .../IpcSharedMemoryCrashDetectionSelfTest.java  |    9 +-
 .../IpcSharedMemoryBenchmarkReader.java         |    2 +-
 .../ignite/logger/java/JavaLoggerTest.java      |    3 +-
 .../OptimizedMarshallerNodeFailoverTest.java    |    4 -
 ...heckpointSpiMultipleDirectoriesSelfTest.java |    6 +-
 .../GridAbstractCommunicationSelfTest.java      |    2 -
 ...mmunicationSpiConcurrentConnectSelfTest.java |   14 +
 ...nicationSpiConcurrentConnectSslSelfTest.java |   35 +
 ...cpCommunicationSpiMultithreadedSelfTest.java |    2 -
 ...GridTcpCommunicationSpiRecoverySelfTest.java |   14 +
 ...dTcpCommunicationSpiRecoverySslSelfTest.java |   35 +
 ...CommunicationSpiSslSmallBuffersSelfTest.java |   43 +
 .../tcp/IgniteCacheSslStartStopSelfTest.java    |    1 -
 .../tcp/TcpDiscoverySnapshotHistoryTest.java    |   14 +-
 .../GridSwapSpaceSpiAbstractSelfTest.java       |    2 -
 .../ignite/testframework/GridTestUtils.java     |   37 +
 .../ignite/testframework/IgniteTestSuite.java   |   18 +-
 .../testframework/junits/GridAbstractTest.java  |    2 +-
 .../junits/GridTestKernalContext.java           |    2 +
 .../ignite/testframework/junits/IgniteMock.java |   10 +
 .../junits/IgniteTestResources.java             |    2 -
 .../junits/spi/GridSpiAbstractTest.java         |    2 -
 .../testsuites/IgniteBinaryBasicTestSuite.java  |    2 -
 .../ignite/testsuites/IgniteCacheTestSuite.java |    4 +
 .../testsuites/IgniteCacheTestSuite4.java       |    2 +
 .../testsuites/IgniteComputeGridTestSuite.java  |    2 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    2 -
 .../IgniteSpiCommunicationSelfTestSuite.java    |   11 +-
 .../CacheNoValueClassOnServerTestClient.java    |    2 -
 .../CacheConfigurationP2PTestClient.java        |    2 -
 .../IgniteHadoopClientProtocolProvider.java     |   53 +-
 .../processors/hadoop/HadoopExternalSplit.java  |    8 +
 .../processors/hadoop/HadoopHelperImpl.java     |   13 +
 .../processors/hadoop/HadoopSplitWrapper.java   |    9 +
 .../processors/hadoop/impl/HadoopUtils.java     |   11 +-
 .../HadoopBasicFileSystemFactoryDelegate.java   |   16 +-
 ...doopIgfsSecondaryFileSystemDelegateImpl.java |    3 +-
 ...HadoopKerberosFileSystemFactoryDelegate.java |    7 +-
 .../hadoop/impl/igfs/HadoopIgfsProperties.java  |    3 +
 .../processors/hadoop/impl/v2/HadoopV2Job.java  |   13 +-
 .../hadoop/impl/v2/HadoopV2TaskContext.java     |    4 +-
 .../hadoop/jobtracker/HadoopJobTracker.java     |   61 +
 .../external/HadoopExternalTaskExecutor.java    |   13 +-
 .../child/HadoopExternalProcessStarter.java     |   10 +-
 .../HadoopExternalCommunication.java            |   10 +-
 .../communication/HadoopMarshallerFilter.java   |   13 +-
 .../hadoop/impl/HadoopAbstractSelfTest.java     |    4 +-
 .../hadoop/impl/HadoopPlannerMockJob.java       |    7 +
 ...opClientProtocolMultipleServersSelfTest.java |  324 ++
 .../impl/igfs/Hadoop1DualAbstractTest.java      |   12 +-
 ...oopSecondaryFileSystemConfigurationTest.java |   12 +
 .../HadoopExternalCommunicationSelfTest.java    |    2 +-
 .../testsuites/IgniteHadoopTestSuite.java       |    2 +
 .../hibernate/CacheHibernateBlobStore.java      |    4 +-
 modules/ignored-tests/pom.xml                   |    8 +
 ...gniteIgnoredBinarySimpleMapperTestSuite.java |   41 +
 .../IgniteIgnoredBinaryTestSuite.java           |   42 +
 .../testsuites/IgniteIgnoredTestSuite.java      |    4 +
 .../processors/query/h2/IgniteH2Indexing.java   |  164 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |   66 +-
 .../query/h2/twostep/GridMergeIndex.java        |   12 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  127 +-
 .../h2/twostep/msg/GridH2QueryRequest.java      |   42 +-
 .../cache/CacheConfigurationP2PTestServer.java  |    2 -
 ...niteCacheDistributedQueryCancelSelfTest.java |  176 +
 ...butedQueryStopOnCancelOrTimeoutSelfTest.java |  253 ++
 ...cheQueryAbstractDistributedJoinSelfTest.java |  290 ++
 ...QueryNodeRestartDistributedJoinSelfTest.java |  262 +-
 .../IgniteCacheQueryNodeRestartSelfTest2.java   |  125 +-
 ...nCancelOrTimeoutDistributedJoinSelfTest.java |  138 +
 .../IgniteCacheReplicatedQuerySelfTest.java     |    2 +
 ...eCacheLocalQueryCancelOrTimeoutSelfTest.java |  158 +
 .../query/IgniteSqlSplitterSelfTest.java        |    2 +
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   13 +-
 .../query/h2/sql/BaseH2CompareQueryTest.java    |    3 +-
 ...narySimpleNameMapperCacheQueryTestSuite.java |    2 -
 .../IgniteCacheQuerySelfTestSuite.java          |   12 +-
 .../IgniteCacheQuerySelfTestSuite2.java         |   13 +-
 .../serialization/CacheEventDeserializer.java   |    3 +-
 .../serialization/CacheEventSerializer.java     |    3 +-
 modules/platforms/cpp/common/Makefile.am        |    4 +-
 .../common/include/ignite/common/concurrent.h   |    5 +-
 .../cpp/common/include/ignite/ignite_error.h    |    4 +-
 .../platforms/cpp/common/src/ignite_error.cpp   |    4 +-
 modules/platforms/cpp/core-test/Makefile.am     |   10 +-
 .../cpp/core-test/config/cache-query.xml        |   12 +-
 .../cpp/core-test/src/cache_query_test.cpp      |    2 +-
 .../cpp/core-test/src/decimal_test.cpp          |    2 +-
 .../cpp/core-test/src/interop_memory_test.cpp   |    5 +-
 .../cpp/core/include/ignite/cache/cache.h       |   15 +-
 .../core/include/ignite/impl/cache/cache_impl.h |   41 +-
 .../ignite/impl/interop/interop_target.h        |   18 +
 .../cpp/core/src/impl/cache/cache_impl.cpp      |   48 +-
 .../core/src/impl/cache/query/query_impl.cpp    |   15 +-
 .../core/src/impl/interop/interop_target.cpp    |   25 +
 .../src/impl/transactions/transactions_impl.cpp |  135 +-
 .../cpp/jni/include/ignite/jni/exports.h        |  106 +-
 .../platforms/cpp/jni/include/ignite/jni/java.h |  229 +-
 modules/platforms/cpp/jni/project/vs/module.def |   88 +-
 modules/platforms/cpp/jni/src/exports.cpp       |  359 +-
 modules/platforms/cpp/jni/src/java.cpp          | 1405 +------
 modules/platforms/cpp/odbc-test/Makefile.am     |    4 +-
 .../cpp/odbc-test/src/queries_test.cpp          |    4 +-
 .../cpp/odbc/src/config/configuration.cpp       |    2 +-
 modules/platforms/cpp/odbc/src/odbc.cpp         |    4 +-
 .../Cache/CacheAbstractTest.cs                  |    2 +-
 .../IgniteConfigurationTest.cs                  |   24 +-
 .../Services/ServicesTest.cs                    |    2 +
 .../Apache.Ignite.Core.csproj                   |    3 +-
 .../Impl/Cache/CacheAffinityImpl.cs             |    5 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |  484 ++-
 .../Apache.Ignite.Core/Impl/Cache/CacheLock.cs  |   24 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheOp.cs    |   43 +-
 .../Impl/Cache/ICacheLockInternal.cs            |   47 +
 .../Impl/Cache/Query/AbstractQueryCursor.cs     |   10 +-
 .../Continuous/ContinuousQueryHandleImpl.cs     |  117 +-
 .../Impl/Cache/Store/CacheStore.cs              |  108 +-
 .../Impl/Cluster/ClusterGroupImpl.cs            |   64 +-
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |   30 +-
 .../Impl/Compute/ComputeImpl.cs                 |   46 +-
 .../Impl/DataStructures/AtomicLong.cs           |   34 +-
 .../Impl/DataStructures/AtomicReference.cs      |    8 +-
 .../Impl/DataStructures/AtomicSequence.cs       |   26 +-
 .../Impl/Datastream/DataStreamerImpl.cs         |   49 +-
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |  180 +-
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |    4 +-
 .../Apache.Ignite.Core/Impl/IgniteUtils.cs      |    2 +-
 .../Impl/Messaging/Messaging.cs                 |  113 +-
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |  233 +-
 .../Impl/Services/Services.cs                   |  146 +-
 .../Impl/Transactions/TransactionsImpl.cs       |   64 +-
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |  272 +-
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |   41 +-
 .../Impl/Unmanaged/UnmanagedUtils.cs            |  553 +--
 .../dotnet/Apache.Ignite.sln.DotSettings        |    3 +-
 .../Apache.Ignite.sln.TeamCity.DotSettings      |    1 +
 .../spi/deployment/uri/UriDeploymentSpi.java    |    2 +-
 .../scala/org/apache/ignite/visor/visor.scala   |    2 +-
 modules/web-console/DEVNOTES.txt                |    2 +-
 modules/web-console/backend/app/agent.js        |   26 +-
 modules/web-console/backend/app/browser.js      |   14 +-
 modules/web-console/backend/app/mongo.js        |  207 +-
 .../backend/routes/demo/domains.json            |   15 +-
 modules/web-console/backend/services/agents.js  |    1 +
 modules/web-console/backend/services/auth.js    |    8 +-
 modules/web-console/backend/services/caches.js  |   14 +-
 .../web-console/backend/services/clusters.js    |   19 +-
 modules/web-console/backend/services/domains.js |   12 +-
 modules/web-console/backend/services/igfss.js   |   12 +-
 .../backend/test/unit/AuthService.test.js       |    4 +-
 .../backend/test/unit/CacheService.test.js      |  135 +-
 .../backend/test/unit/ClusterService.test.js    |  132 +-
 .../backend/test/unit/DomainService.test.js     |  144 +-
 .../backend/test/unit/IgfsService.test.js       |  122 +-
 modules/web-console/frontend/.eslintrc          |    6 +-
 modules/web-console/frontend/app/app.config.js  |   10 +
 modules/web-console/frontend/app/app.js         |   37 +-
 .../web-console/frontend/app/data/dialects.json |    9 +
 .../frontend/app/data/java-classes.json         |    4 +-
 .../frontend/app/decorator/tooltip.js           |   13 +-
 .../app/directives/on-focus-out.directive.js    |   37 +
 .../directives/restore-input-focus.directive.js |   24 +
 .../ui-ace-java/ui-ace-java.controller.js       |   78 +-
 .../ui-ace-java/ui-ace-java.directive.js        |  100 +-
 .../ui-ace-pojos/ui-ace-pojos.controller.js     |    2 +-
 .../ui-ace-pom/ui-ace-pom.controller.js         |    2 +-
 .../ui-ace-sharp/ui-ace-sharp.controller.js     |   32 +
 .../ui-ace-sharp/ui-ace-sharp.directive.js      |  133 +
 .../directives/ui-ace-sharp/ui-ace-sharp.jade   |   22 +
 .../ui-ace-spring/ui-ace-spring.controller.js   |   88 +
 .../ui-ace-spring/ui-ace-spring.directive.js    |   66 +
 .../directives/ui-ace-spring/ui-ace-spring.jade |   17 +
 .../ui-ace-xml/ui-ace-xml.controller.js         |   27 -
 .../ui-ace-xml/ui-ace-xml.directive.js          |  147 -
 .../app/directives/ui-ace-xml/ui-ace-xml.jade   |   17 -
 .../ui-grid-settings/ui-grid-settings.jade      |   33 +
 .../ui-grid-settings/ui-grid-settings.scss      |   38 +
 .../frontend/app/filters/default-name.filter.js |   21 +
 .../frontend/app/filters/hasPojo.filter.js      |    5 +-
 .../helpers/jade/form/form-field-feedback.jade  |    5 +-
 .../frontend/app/helpers/jade/mixins.jade       |  219 +-
 .../frontend/app/modules/agent/agent.module.js  |   10 +-
 .../modules/configuration/Version.service.js    |   82 +-
 .../configuration/configuration.module.js       |   30 +-
 .../generator/AbstractTransformer.js            |  341 ++
 .../modules/configuration/generator/Beans.js    |  379 ++
 .../generator/ConfigurationGenerator.js         | 1785 +++++++++
 .../configuration/generator/Java.service.js     |   21 -
 .../generator/JavaTransformer.service.js        | 1721 +++++++++
 .../generator/PlatformGenerator.js              |  522 +++
 .../configuration/generator/Pom.service.js      |  173 +-
 .../generator/Properties.service.js             |   74 +
 .../configuration/generator/Readme.service.js   |   79 +
 .../generator/SharpTransformer.service.js       |  243 ++
 .../generator/SpringTransformer.service.js      |  325 ++
 .../configuration/generator/StringBuilder.js    |   76 +
 .../configuration/generator/Xml.service.js      |   21 -
 .../defaults/cache.platform.provider.js         |   60 +
 .../generator/defaults/cache.provider.js        |  129 +
 .../defaults/cluster.platform.provider.js       |   49 +
 .../generator/defaults/cluster.provider.js      |  293 ++
 .../generator/defaults/igfs.provider.js         |   68 +
 .../configuration/generator/generator-common.js |  625 +++
 .../configuration/generator/generator-java.js   | 3617 ++++++++++++++++++
 .../generator/generator-optional.js             |   25 +
 .../configuration/generator/generator-spring.js | 2111 ++++++++++
 .../app/modules/form/field/input/select.scss    |   21 -
 .../app/modules/form/field/input/text.scss      |    1 -
 .../frontend/app/modules/form/form.module.js    |   10 +-
 .../app/modules/form/panel/revert.directive.js  |    4 +-
 .../modules/form/services/FormGUID.service.js   |   22 +
 .../validator/java-built-in-class.directive.js  |    6 +-
 .../form/validator/java-identifier.directive.js |    6 +-
 .../form/validator/java-keywords.directive.js   |   15 +-
 .../java-package-specified.directive.js         |    6 +-
 .../frontend/app/modules/nodes/Nodes.service.js |   69 +
 .../modules/nodes/nodes-dialog.controller.js    |   68 +
 .../app/modules/nodes/nodes-dialog.jade         |   35 +
 .../app/modules/nodes/nodes-dialog.scss         |   37 +
 .../frontend/app/modules/nodes/nodes.module.js  |   27 +
 .../frontend/app/modules/sql/sql.controller.js  |  269 +-
 .../app/modules/states/configuration.state.js   |    2 +-
 .../configuration/caches/client-near-cache.jade |   50 +
 .../states/configuration/caches/general.jade    |    3 +
 .../states/configuration/caches/memory.jade     |   14 +-
 .../configuration/caches/near-cache-client.jade |   51 +
 .../configuration/caches/near-cache-server.jade |   52 +
 .../configuration/caches/node-filter.jade       |   49 -
 .../states/configuration/caches/query.jade      |   30 +-
 .../states/configuration/caches/rebalance.jade  |    3 +-
 .../configuration/caches/server-near-cache.jade |   51 -
 .../states/configuration/caches/store.jade      |   84 +-
 .../states/configuration/clusters/atomic.jade   |    3 +-
 .../configuration/clusters/attributes.jade      |    4 +-
 .../states/configuration/clusters/binary.jade   |   28 +-
 .../configuration/clusters/cache-key-cfg.jade   |    9 +-
 .../configuration/clusters/checkpoint.jade      |   85 +
 .../configuration/clusters/checkpoint/fs.jade   |   66 +
 .../configuration/clusters/checkpoint/jdbc.jade |   45 +
 .../configuration/clusters/checkpoint/s3.jade   |  174 +
 .../configuration/clusters/collision.jade       |   13 +-
 .../clusters/collision/job-stealing.jade        |    2 +-
 .../configuration/clusters/communication.jade   |    3 +-
 .../configuration/clusters/connector.jade       |    3 +-
 .../configuration/clusters/deployment.jade      |    3 +-
 .../configuration/clusters/discovery.jade       |    3 +-
 .../states/configuration/clusters/events.jade   |   37 +-
 .../states/configuration/clusters/failover.jade |   19 +-
 .../states/configuration/clusters/general.jade  |    3 +
 .../clusters/general/discovery/zookeeper.jade   |    2 +
 .../states/configuration/clusters/igfs.jade     |    3 +-
 .../configuration/clusters/load-balancing.jade  |  104 +
 .../configuration/clusters/marshaller.jade      |    3 +-
 .../states/configuration/clusters/odbc.jade     |    3 +-
 .../states/configuration/clusters/ssl.jade      |    7 +-
 .../states/configuration/clusters/swap.jade     |    3 +-
 .../configuration/clusters/transactions.jade    |    6 +-
 .../states/configuration/domains/general.jade   |   12 +-
 .../states/configuration/domains/query.jade     |   40 +-
 .../states/configuration/domains/store.jade     |   15 +-
 .../states/configuration/igfs/general.jade      |    3 +
 .../modules/states/configuration/igfs/ipc.jade  |    2 +-
 .../modules/states/configuration/igfs/misc.jade |    4 +-
 .../states/configuration/igfs/secondary.jade    |    3 +-
 .../configuration/summary/summary.controller.js |  106 +-
 .../app/services/ErrorPopover.service.js        |    2 +-
 .../frontend/app/services/FormUtils.service.js  |    6 +-
 .../frontend/app/services/JavaTypes.service.js  |   70 +-
 .../app/services/LegacyTable.service.js         |   24 +-
 .../app/services/LegacyUtils.service.js         |  128 +-
 .../frontend/app/services/SqlTypes.service.js   |    6 +-
 modules/web-console/frontend/app/vendor.js      |    3 +-
 .../frontend/controllers/caches-controller.js   |  121 +-
 .../frontend/controllers/clusters-controller.js |  177 +-
 .../frontend/controllers/domains-controller.js  |  148 +-
 .../frontend/controllers/igfs-controller.js     |    6 +-
 .../frontend/generator/generator-common.js      |  625 ---
 .../frontend/generator/generator-java.js        | 3611 -----------------
 .../frontend/generator/generator-optional.js    |   25 -
 .../frontend/generator/generator-properties.js  |  175 -
 .../frontend/generator/generator-readme.js      |   85 -
 .../frontend/generator/generator-xml.js         | 2108 ----------
 .../frontend/gulpfile.babel.js/paths.js         |   31 +-
 .../frontend/gulpfile.babel.js/tasks/jade.js    |   21 +-
 .../frontend/gulpfile.babel.js/tasks/watch.js   |    6 +-
 .../gulpfile.babel.js/webpack/common.js         |    2 +-
 .../webpack/environments/development.js         |   17 +-
 .../webpack/environments/test.js                |   52 +
 .../frontend/gulpfile.babel.js/webpack/index.js |    4 +-
 modules/web-console/frontend/package.json       |   16 +-
 .../frontend/public/stylesheets/style.scss      |   75 +-
 .../frontend/test/karma.conf.babel.js           |   91 +
 modules/web-console/frontend/test/karma.conf.js |   98 +-
 .../frontend/test/unit/JavaTransformer.test.js  |   57 +
 .../frontend/test/unit/JavaTypes.test.js        |   23 +-
 .../frontend/test/unit/SharpTransformer.test.js |   55 +
 .../test/unit/SpringTransformer.test.js         |   57 +
 .../frontend/test/unit/SqlTypes.test.js         |   17 -
 .../frontend/test/unit/Version.test.js          |   82 +
 .../test/unit/defaultName.filter.test.js        |   38 +
 .../frontend/views/configuration/caches.jade    |    3 +-
 .../frontend/views/configuration/clusters.jade  |    3 +-
 .../views/configuration/domains-import.jade     |   14 +-
 .../summary-project-structure.jade              |    2 +-
 .../frontend/views/configuration/summary.jade   |   57 +-
 .../frontend/views/sql/cache-metadata.jade      |    2 +-
 .../frontend/views/sql/notebook-new.jade        |    4 +-
 modules/web-console/frontend/views/sql/sql.jade |   80 +-
 .../views/templates/agent-download.jade         |    6 +-
 .../frontend/views/templates/batch-confirm.jade |    4 +-
 .../frontend/views/templates/clone.jade         |    4 +-
 .../frontend/views/templates/confirm.jade       |    4 +-
 .../frontend/views/templates/demo-info.jade     |    4 +-
 .../views/templates/getting-started.jade        |    4 +-
 .../frontend/views/templates/message.jade       |    4 +-
 .../frontend/views/templates/select.jade        |    2 +-
 .../ignite/console/demo/AgentClusterDemo.java   |    2 +-
 .../ignite/cache/websession/WebSessionV2.java   |   26 +-
 501 files changed, 26906 insertions(+), 14876 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcResultSetSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
----------------------------------------------------------------------

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 711453c,fd13e98..435fcfb
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@@ -81,9 -81,9 +81,10 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.cache.GridCacheStopSelfTest;
  import org.apache.ignite.internal.processors.cache.GridCacheStorePutxSelfTest;
  import org.apache.ignite.internal.processors.cache.GridCacheStoreValueBytesSelfTest;
 +import org.apache.ignite.internal.processors.cache.GridCacheSwapCleanupTest;
  import org.apache.ignite.internal.processors.cache.GridCacheSwapPreloadSelfTest;
  import org.apache.ignite.internal.processors.cache.GridCacheSwapReloadSelfTest;
+ import org.apache.ignite.internal.processors.cache.GridCacheTtlManagerEvictionSelfTest;
  import org.apache.ignite.internal.processors.cache.GridCacheTtlManagerSelfTest;
  import org.apache.ignite.internal.processors.cache.GridCacheTxPartitionedLocalStoreSelfTest;
  import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicInvokeTest;

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
index 145a896,d9cc8c0..2d06f3a
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
@@@ -66,10 -66,8 +66,9 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.service.IgniteServiceDeploymentClassLoadingDefaultMarshallerTest;
  import org.apache.ignite.internal.processors.service.IgniteServiceDeploymentClassLoadingJdkMarshallerTest;
  import org.apache.ignite.internal.processors.service.IgniteServiceDeploymentClassLoadingOptimizedMarshallerTest;
 +import org.apache.ignite.internal.processors.service.IgniteServiceDynamicCachesSelfTest;
  import org.apache.ignite.internal.processors.service.IgniteServiceReassignmentTest;
  import org.apache.ignite.internal.processors.service.ServicePredicateAccessCacheTest;
- import org.apache.ignite.internal.util.GridStartupWithSpecifiedWorkDirectorySelfTest;
  import org.apache.ignite.internal.util.GridStartupWithUndefinedIgniteHomeSelfTest;
  import org.apache.ignite.spi.communication.GridCacheMessageSelfTest;
  import org.apache.ignite.testframework.GridTestUtils;

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java
index 048bc8c,a673a73..6d9cbb1
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java
@@@ -44,9 -48,14 +44,10 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.util.typedef.X;
  import org.apache.ignite.internal.util.typedef.internal.S;
  import org.apache.ignite.internal.util.typedef.internal.U;
 -import org.apache.ignite.lang.IgniteBiTuple;
  import org.apache.ignite.lang.IgnitePredicate;
  import org.apache.ignite.testframework.GridTestUtils;
+ import org.apache.ignite.testsuites.IgniteIgnore;
  import org.apache.ignite.transactions.Transaction;
 -import org.springframework.util.ReflectionUtils;
  
  import static org.apache.ignite.cache.CacheMode.REPLICATED;
  import static org.apache.ignite.cache.CachePeekMode.ALL;

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
----------------------------------------------------------------------
diff --cc modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
index e623969,fcc1652..9dc5b1e
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
@@@ -1,19 -1,19 +1,19 @@@
 -\ufeff\ufeff/*
 +\ufeff/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *      http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
  
  namespace Apache.Ignite.Core.Impl.Unmanaged
  {

http://git-wip-us.apache.org/repos/asf/ignite/blob/173e2b83/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings
----------------------------------------------------------------------
diff --cc modules/platforms/dotnet/Apache.Ignite.sln.DotSettings
index 688a924,72ce015..078e9fb
--- a/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings
+++ b/modules/platforms/dotnet/Apache.Ignite.sln.DotSettings
@@@ -1,9 -1,6 +1,10 @@@
  \ufeff<wpf:ResourceDictionary xml:space="preserve" xmlns:x="http://schemas.microsoft.com/winfx/2006/xaml" xmlns:s="clr-namespace:System;assembly=mscorlib" xmlns:ss="urn:shemas-jetbrains-com:settings-storage-xaml" xmlns:wpf="http://schemas.microsoft.com/winfx/2006/xaml/presentation">
  	<s:String x:Key="/Default/CodeInspection/CSharpLanguageProject/LanguageLevel/@EntryValue">CSharp50</s:String>
 +	<s:String x:Key="/Default/CodeStyle/CodeCleanup/Profiles/=silent/@EntryIndexedValue">&lt;?xml version="1.0" encoding="utf-16"?&gt;&lt;Profile name="silent"&gt;&lt;CSReorderTypeMembers&gt;True&lt;/CSReorderTypeMembers&gt;&lt;CSCodeStyleAttributes ArrangeTypeAccessModifier="True" ArrangeTypeMemberAccessModifier="True" SortModifiers="True" RemoveRedundantParentheses="False" AddMissingParentheses="False" ArrangeBraces="False" ArrangeAttributes="False" ArrangeArgumentsStyle="False" /&gt;&lt;RemoveCodeRedundancies&gt;True&lt;/RemoveCodeRedundancies&gt;&lt;CSArrangeQualifiers&gt;True&lt;/CSArrangeQualifiers&gt;&lt;CSOptimizeUsings&gt;&lt;OptimizeUsings&gt;True&lt;/OptimizeUsings&gt;&lt;EmbraceInRegion&gt;False&lt;/EmbraceInRegion&gt;&lt;RegionName&gt;&lt;/RegionName&gt;&lt;/CSOptimizeUsings&gt;&lt;CSShortenReferences&gt;True&lt;/CSShortenReferences&gt;&lt;CSReformatCode&gt;True&lt;/CSReformatCode&gt;&lt;/Profile&gt;</s:String>
 +	<s:String x:Key="/Default/CodeStyle/CodeCleanup/SilentCleanupProfile/@EntryValue">silent</s:String>
 +	<s:String x:Key="/Default/CodeStyle/CodeFormatting/XmlDocFormatter/IndentSubtags/@EntryValue">DoNotTouch</s:String>
 +	<s:String x:Key="/Default/CodeStyle/CodeFormatting/XmlDocFormatter/IndentTagContent/@EntryValue">DoNotTouch</s:String>
- 	<s:Boolean x:Key="/Default/CodeStyle/CSharpUsing/AddImportsToDeepestScope/@EntryValue">True</s:Boolean>	
+ 	<s:Boolean x:Key="/Default/CodeStyle/CSharpUsing/AddImportsToDeepestScope/@EntryValue">True</s:Boolean>
  	<s:Boolean x:Key="/Default/CodeStyle/CSharpUsing/QualifiedUsingAtNestedScope/@EntryValue">True</s:Boolean>
+ 	<s:String x:Key="/Default/CodeInspection/Highlighting/InspectionSeverities/=ConvertClosureToMethodGroup/@EntryIndexedValue">DO_NOT_SHOW</s:String>
  </wpf:ResourceDictionary>


[45/50] [abbrv] ignite git commit: IGNITE-4126 .NET: Add IgniteConfiguration.SwapSpaceSpi

Posted by vo...@apache.org.
IGNITE-4126 .NET: Add IgniteConfiguration.SwapSpaceSpi


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

Branch: refs/heads/ignite-2693
Commit: d69e26dd8d4fd9383a149c93c251911a8dd67528
Parents: c6921a3
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Nov 11 12:22:55 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Nov 11 12:22:55 2016 +0300

----------------------------------------------------------------------
 .../utils/PlatformConfigurationUtils.java       |  61 ++++++++--
 .../Apache.Ignite.Core.Tests.csproj             |   1 +
 .../Cache/CacheSwapSpaceTest.cs                 | 120 +++++++++++++++++++
 .../IgniteConfigurationSerializerTest.cs        |  20 +++-
 .../IgniteConfigurationTest.cs                  |  22 +++-
 .../Apache.Ignite.Core.csproj                   |   3 +
 .../Cache/Configuration/CacheConfiguration.cs   |   2 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  15 ++-
 .../IgniteConfigurationSection.xsd              |  37 ++++++
 .../Impl/SwapSpace/SwapSpaceSerializer.cs       |  99 +++++++++++++++
 .../SwapSpace/File/FileSwapSpaceSpi.cs          | 105 ++++++++++++++++
 .../SwapSpace/ISwapSpaceSpi.cs                  |  35 ++++++
 12 files changed, 509 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index 26374f0..9480dae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -51,6 +51,9 @@ 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.multicast.TcpDiscoveryMulticastIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.spi.swapspace.SwapSpaceSpi;
+import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi;
+import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpiMBean;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
 
@@ -66,8 +69,14 @@ import java.util.Map;
 /**
  * Configuration utils.
  */
-@SuppressWarnings("unchecked")
+@SuppressWarnings({"unchecked", "TypeMayBeWeakened"})
 public class PlatformConfigurationUtils {
+    /** */
+    private static final byte SWAP_TYP_NONE = 0;
+
+    /** */
+    private static final byte SWAP_TYP_FILE = 1;
+
     /**
      * Write .Net configuration to the stream.
      *
@@ -202,7 +211,7 @@ public class PlatformConfigurationUtils {
      * @param in Stream.
      * @return Eviction policy.
      */
-    public static EvictionPolicy readEvictionPolicy(BinaryRawReader in) {
+    private static EvictionPolicy readEvictionPolicy(BinaryRawReader in) {
         byte plcTyp = in.readByte();
 
         switch (plcTyp) {
@@ -277,7 +286,7 @@ public class PlatformConfigurationUtils {
      * @param cfg NearCacheConfiguration.
      */
     @SuppressWarnings("TypeMayBeWeakened")
-    public static void writeNearConfiguration(BinaryRawWriter out, NearCacheConfiguration cfg) {
+    private static void writeNearConfiguration(BinaryRawWriter out, NearCacheConfiguration cfg) {
         assert cfg != null;
 
         out.writeInt(cfg.getNearStartSize());
@@ -371,7 +380,7 @@ public class PlatformConfigurationUtils {
      * @param in Stream.
      * @return QueryEntity.
      */
-    public static QueryEntity readQueryEntity(BinaryRawReader in) {
+    private static QueryEntity readQueryEntity(BinaryRawReader in) {
         QueryEntity res = new QueryEntity();
 
         res.setKeyType(in.readString());
@@ -422,7 +431,7 @@ public class PlatformConfigurationUtils {
      * @param in Reader.
      * @return Query index.
      */
-    public static QueryIndex readQueryIndex(BinaryRawReader in) {
+    private static QueryIndex readQueryIndex(BinaryRawReader in) {
         QueryIndex res = new QueryIndex();
 
         res.setName(in.readString());
@@ -529,6 +538,27 @@ public class PlatformConfigurationUtils {
 
             cfg.setTransactionConfiguration(tx);
         }
+
+        byte swapType = in.readByte();
+
+        switch (swapType) {
+            case SWAP_TYP_FILE: {
+                FileSwapSpaceSpi swap = new FileSwapSpaceSpi();
+
+                swap.setBaseDirectory(in.readString());
+                swap.setMaximumSparsity(in.readFloat());
+                swap.setMaxWriteQueueSize(in.readInt());
+                swap.setReadStripesNumber(in.readInt());
+                swap.setWriteBufferSize(in.readInt());
+
+                cfg.setSwapSpaceSpi(swap);
+
+                break;
+            }
+
+            default:
+                assert swapType == SWAP_TYP_NONE;
+        }
     }
 
     /**
@@ -537,7 +567,7 @@ public class PlatformConfigurationUtils {
      * @param cfg IgniteConfiguration to update.
      * @param in Reader.
      */
-    public static void readCacheConfigurations(BinaryRawReaderEx in, IgniteConfiguration cfg) {
+    private static void readCacheConfigurations(BinaryRawReaderEx in, IgniteConfiguration cfg) {
         int len = in.readInt();
 
         if (len == 0)
@@ -569,7 +599,7 @@ public class PlatformConfigurationUtils {
      * @param cfg IgniteConfiguration to update.
      * @param in Reader.
      */
-    public static void readDiscoveryConfiguration(BinaryRawReader in, IgniteConfiguration cfg) {
+    private static void readDiscoveryConfiguration(BinaryRawReader in, IgniteConfiguration cfg) {
         boolean hasConfig = in.readBoolean();
 
         if (!hasConfig)
@@ -917,6 +947,23 @@ public class PlatformConfigurationUtils {
         else
             w.writeBoolean(false);
 
+        SwapSpaceSpi swap = cfg.getSwapSpaceSpi();
+
+        if (swap instanceof FileSwapSpaceSpiMBean) {
+            w.writeByte(SWAP_TYP_FILE);
+
+            FileSwapSpaceSpiMBean fileSwap = (FileSwapSpaceSpiMBean)swap;
+
+            w.writeString(fileSwap.getBaseDirectory());
+            w.writeFloat(fileSwap.getMaximumSparsity());
+            w.writeInt(fileSwap.getMaxWriteQueueSize());
+            w.writeInt(fileSwap.getReadStripesNumber());
+            w.writeInt(fileSwap.getWriteBufferSize());
+        }
+        else {
+            w.writeByte(SWAP_TYP_NONE);
+        }
+
         w.writeString(cfg.getIgniteHome());
 
         w.writeLong(ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getInit());

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 34d93d7..a9de399 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -67,6 +67,7 @@
     <Compile Include="Cache\Affinity\AffinityKeyTest.cs" />
     <Compile Include="Cache\Affinity\AffinityTopologyVersionTest.cs" />
     <Compile Include="Cache\CacheResultTest.cs" />
+    <Compile Include="Cache\CacheSwapSpaceTest.cs" />
     <Compile Include="Cache\Store\CacheStoreAdapterTest.cs" />
     <Compile Include="Collections\MultiValueDictionaryTest.cs" />
     <Compile Include="Collections\ReadOnlyCollectionTest.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheSwapSpaceTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheSwapSpaceTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheSwapSpaceTest.cs
new file mode 100644
index 0000000..296fd13
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheSwapSpaceTest.cs
@@ -0,0 +1,120 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Cache
+{
+    using System.IO;
+    using System.Linq;
+    using System.Threading;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Cache.Eviction;
+    using Apache.Ignite.Core.Discovery.Tcp;
+    using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.SwapSpace.File;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests the swap space.
+    /// </summary>
+    public class CacheSwapSpaceTest
+    {
+        /** */
+        private readonly string _tempDir = IgniteUtils.GetTempDirectoryName();
+
+        /// <summary>
+        /// Fixture tear down.
+        /// </summary>
+        [TestFixtureTearDown]
+        public void FixtureTearDown()
+        {
+            Ignition.StopAll(true);
+
+            Directory.Delete(_tempDir, true);
+        }
+
+        /// <summary>
+        /// Tests that swap space is disabled by default and cache can't have EnableSwap.
+        /// </summary>
+        [Test]
+        public void TestDisabledSwapSpace()
+        {
+            var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration());
+
+            using (var ignite = Ignition.Start(cfg))
+            {
+                // NoopSwapSpaceSpi is used by default.
+                Assert.IsNull(ignite.GetConfiguration().SwapSpaceSpi);
+
+                var ex = Assert.Throws<CacheException>(
+                    () => ignite.CreateCache<int, int>(new CacheConfiguration {EnableSwap = true}));
+
+                Assert.IsTrue(ex.Message.EndsWith("has not swap SPI configured"));
+            }
+        }
+
+        /// <summary>
+        /// Tests the swap space.
+        /// </summary>
+        [Test]
+        public void TestSwapSpace()
+        {
+            const int entrySize = 1024;
+
+            var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                SwapSpaceSpi = new FileSwapSpaceSpi
+                {
+                    BaseDirectory = _tempDir,
+                    WriteBufferSize = 64
+                }
+            };
+
+            using (var ignite = Ignition.Start(cfg))
+            {
+                // Create cache with eviction and swap.
+                var cache = ignite.CreateCache<int, byte[]>(new CacheConfiguration("cache")
+                {
+                    EnableSwap = true,
+                    EvictionPolicy = new LruEvictionPolicy
+                    {
+                        MaxSize = 3
+                    },
+                    OffHeapMaxMemory = 5 * entrySize
+                });
+
+                // Populate to trigger eviction.
+                var data = Enumerable.Range(1, entrySize).Select(x => (byte) x).ToArray();
+
+                for (int i = 0; i < 10; i++)
+                    cache[i] = data;
+
+                // Check that swap files exist.
+                var files = Directory.GetFiles(_tempDir, "*.*", SearchOption.AllDirectories);
+                CollectionAssert.IsNotEmpty(files);
+                
+                // Wait for metrics update and check metrics.
+                Thread.Sleep(((TcpDiscoverySpi) ignite.GetConfiguration().DiscoverySpi).HeartbeatFrequency);
+
+                var metrics = cache.GetMetrics();
+
+                Assert.AreEqual(4, metrics.OffHeapEntriesCount);  // Entry takes more space than the value
+                Assert.AreEqual(3, metrics.OverflowSize / entrySize);  // 10 - 3 - 4 = 3
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
index 9fb5bbd..ac3e553 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -45,6 +45,7 @@ namespace Apache.Ignite.Core.Tests
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Lifecycle;
     using Apache.Ignite.Core.Log;
+    using Apache.Ignite.Core.SwapSpace.File;
     using Apache.Ignite.Core.Tests.Binary;
     using Apache.Ignite.Core.Transactions;
     using Apache.Ignite.NLog;
@@ -113,6 +114,7 @@ namespace Apache.Ignite.Core.Tests
                             <atomicConfiguration backups='2' cacheMode='Local' atomicSequenceReserveSize='250' />
                             <transactionConfiguration defaultTransactionConcurrency='Optimistic' defaultTransactionIsolation='RepeatableRead' defaultTimeout='0:1:2' pessimisticTransactionLogSize='15' pessimisticTransactionLogLinger='0:0:33' />
                             <logger type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+TestLogger, Apache.Ignite.Core.Tests' />
+                            <swapSpaceSpi type='FileSwapSpaceSpi' baseDirectory='abcd' maximumSparsity='0.7' maximumWriteQueueSize='25' readStripesNumber='36' writeBufferSize='47' />
                         </igniteConfig>";
             var reader = XmlReader.Create(new StringReader(xml));
 
@@ -199,6 +201,14 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(new TimeSpan(0, 1, 2), comm.IdleConnectionTimeout);
 
             Assert.IsInstanceOf<TestLogger>(cfg.Logger);
+
+            var swap = cfg.SwapSpaceSpi as FileSwapSpaceSpi;
+            Assert.IsNotNull(swap);
+            Assert.AreEqual("abcd", swap.BaseDirectory);
+            Assert.AreEqual(0.7f, swap.MaximumSparsity);
+            Assert.AreEqual(25, swap.MaximumWriteQueueSize);
+            Assert.AreEqual(36, swap.ReadStripesNumber);
+            Assert.AreEqual(47, swap.WriteBufferSize);
         }
 
         /// <summary>
@@ -599,7 +609,15 @@ namespace Apache.Ignite.Core.Tests
                 IsLateAffinityAssignment = false,
                 SpringConfigUrl = "test",
                 Logger = new IgniteNLogLogger(),
-                FailureDetectionTimeout = TimeSpan.FromMinutes(2)
+                FailureDetectionTimeout = TimeSpan.FromMinutes(2),
+                SwapSpaceSpi = new FileSwapSpaceSpi
+                {
+                    MaximumSparsity = 0.1f,
+                    MaximumWriteQueueSize = 55,
+                    WriteBufferSize = 66,
+                    ReadStripesNumber = 77,
+                    BaseDirectory = "test"
+                }
             };
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
index 30bae47..2e39b9b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -34,6 +34,7 @@ namespace Apache.Ignite.Core.Tests
     using Apache.Ignite.Core.Discovery.Tcp.Static;
     using Apache.Ignite.Core.Events;
     using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.SwapSpace.File;
     using Apache.Ignite.Core.Transactions;
     using NUnit.Framework;
 
@@ -79,6 +80,7 @@ namespace Apache.Ignite.Core.Tests
             CheckDefaultValueAttributes(new LruEvictionPolicy());
             CheckDefaultValueAttributes(new AtomicConfiguration());
             CheckDefaultValueAttributes(new TransactionConfiguration());
+            CheckDefaultValueAttributes(new FileSwapSpaceSpi());
         }
 
         /// <summary>
@@ -171,6 +173,14 @@ namespace Apache.Ignite.Core.Tests
                 Assert.AreEqual(com.UnacknowledgedMessagesBufferSize, resCom.UnacknowledgedMessagesBufferSize);
 
                 Assert.AreEqual(cfg.FailureDetectionTimeout, resCfg.FailureDetectionTimeout);
+
+                var swap = (FileSwapSpaceSpi) cfg.SwapSpaceSpi;
+                var resSwap = (FileSwapSpaceSpi) resCfg.SwapSpaceSpi;
+                Assert.AreEqual(swap.MaximumSparsity, resSwap.MaximumSparsity);
+                Assert.AreEqual(swap.BaseDirectory, resSwap.BaseDirectory);
+                Assert.AreEqual(swap.MaximumWriteQueueSize, resSwap.MaximumWriteQueueSize);
+                Assert.AreEqual(swap.ReadStripesNumber, resSwap.ReadStripesNumber);
+                Assert.AreEqual(swap.WriteBufferSize, resSwap.WriteBufferSize);
             }
         }
 
@@ -398,7 +408,7 @@ namespace Apache.Ignite.Core.Tests
         {
             var props = obj.GetType().GetProperties();
 
-            foreach (var prop in props.Where(p => p.Name != "SelectorsCount"))
+            foreach (var prop in props.Where(p => p.Name != "SelectorsCount" && p.Name != "ReadStripesNumber"))
             {
                 var attr = prop.GetCustomAttributes(true).OfType<DefaultValueAttribute>().FirstOrDefault();
                 var propValue = prop.GetValue(obj, null);
@@ -495,7 +505,15 @@ namespace Apache.Ignite.Core.Tests
                     SocketSendBufferSize = 2045,
                     UnacknowledgedMessagesBufferSize = 3450
                 },
-                FailureDetectionTimeout = TimeSpan.FromSeconds(3.5)
+                FailureDetectionTimeout = TimeSpan.FromSeconds(3.5),
+                SwapSpaceSpi = new FileSwapSpaceSpi
+                {
+                    ReadStripesNumber = 64,
+                    MaximumWriteQueueSize = 8,
+                    WriteBufferSize = 9,
+                    BaseDirectory = Path.GetTempPath(),
+                    MaximumSparsity = 11.22f
+                }
             };
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 66253f6..64d7388 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -418,6 +418,7 @@
     <Compile Include="Impl\Services\ServiceProxyInvoker.cs" />
     <Compile Include="Impl\Services\ServiceProxySerializer.cs" />
     <Compile Include="Impl\Services\Services.cs" />
+    <Compile Include="Impl\SwapSpace\SwapSpaceSerializer.cs" />
     <Compile Include="Impl\Transactions\Transaction.cs" />
     <Compile Include="Impl\Transactions\TransactionImpl.cs" />
     <Compile Include="Impl\Transactions\TransactionMetricsImpl.cs" />
@@ -471,6 +472,8 @@
     <Compile Include="Services\Package-Info.cs" />
     <Compile Include="Services\ServiceConfiguration.cs" />
     <Compile Include="Services\ServiceInvocationException.cs" />
+    <Compile Include="SwapSpace\File\FileSwapSpaceSpi.cs" />
+    <Compile Include="SwapSpace\ISwapSpaceSpi.cs" />
     <Compile Include="Transactions\ITransaction.cs" />
     <Compile Include="Transactions\ITransactionMetrics.cs" />
     <Compile Include="Transactions\ITransactions.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
index cb15564..c0c4722 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
@@ -507,6 +507,8 @@ namespace Apache.Ignite.Core.Cache.Configuration
 
         /// <summary>
         /// Flag indicating whether Ignite should use swap storage by default.
+        /// <para />
+        /// Enabling this requires configured <see cref="IgniteConfiguration.SwapSpaceSpi"/>.
         /// </summary>
         [DefaultValue(DefaultEnableSwap)]
         public bool EnableSwap { get; set; }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
index 68433f7..e7aa64e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
@@ -39,8 +39,10 @@
     using Apache.Ignite.Core.Impl;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Impl.SwapSpace;
     using Apache.Ignite.Core.Lifecycle;
     using Apache.Ignite.Core.Log;
+    using Apache.Ignite.Core.SwapSpace;
     using Apache.Ignite.Core.Transactions;
     using BinaryReader = Apache.Ignite.Core.Impl.Binary.BinaryReader;
     using BinaryWriter = Apache.Ignite.Core.Impl.Binary.BinaryWriter;
@@ -297,10 +299,13 @@
                 writer.WriteInt((int) TransactionConfiguration.DefaultTransactionConcurrency);
                 writer.WriteInt((int) TransactionConfiguration.DefaultTransactionIsolation);
                 writer.WriteLong((long) TransactionConfiguration.DefaultTimeout.TotalMilliseconds);
-                writer.WriteLong((int) TransactionConfiguration.PessimisticTransactionLogLinger.TotalMilliseconds);
+                writer.WriteInt((int) TransactionConfiguration.PessimisticTransactionLogLinger.TotalMilliseconds);
             }
             else
                 writer.WriteBoolean(false);
+
+            // Swap space
+            SwapSpaceSerializer.Write(writer, SwapSpaceSpi);
         }
 
         /// <summary>
@@ -386,6 +391,9 @@
                     PessimisticTransactionLogLinger = TimeSpan.FromMilliseconds(r.ReadInt())
                 };
             }
+
+            // Swap
+            SwapSpaceSpi = SwapSpaceSerializer.Read(r);
         }
 
         /// <summary>
@@ -790,5 +798,10 @@
             get { return _failureDetectionTimeout ?? DefaultFailureDetectionTimeout; }
             set { _failureDetectionTimeout = value; }
         }
+
+        /// <summary>
+        /// Gets or sets the swap space SPI.
+        /// </summary>
+        public ISwapSpaceSpi SwapSpaceSpi { get; set; }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
index e9fb991..06541da 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -1107,6 +1107,43 @@
                         </xs:attribute>
                     </xs:complexType>
                 </xs:element>
+                <xs:element name="swapSpaceSpi" minOccurs="0">
+                    <xs:annotation>
+                        <xs:documentation>Swap space SPI.</xs:documentation>
+                    </xs:annotation>
+                    <xs:complexType>
+                        <xs:attribute name="type" type="xs:string" use="required">
+                            <xs:annotation>
+                                <xs:documentation>Assembly-qualified type name.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="baseDirectory" type="xs:string">
+                            <xs:annotation>
+                                <xs:documentation>Base directory.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="maximumSparsity" type="xs:decimal">
+                            <xs:annotation>
+                                <xs:documentation>Maximum sparsity. This property defines maximum acceptable wasted file space to whole file size ratio. When this ratio becomes higher than specified number compacting thread starts working.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="maximumWriteQueueSize" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Maximum size of the write queue in bytes. If there are more values are waiting to be written to disk then specified size, SPI will block on write operation.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="readStripesNumber" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Read stripes number. Defines number of file channels to be used concurrently. Defaults to Environment.ProcessorCount.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="writeBufferSize" type="xs:int">
+                            <xs:annotation>
+                                <xs:documentation>Size of the write buffer, in bytes. Write to disk occurs only when this buffer is full.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                    </xs:complexType>
+                </xs:element>
             </xs:all>
             <xs:attribute name="gridName" type="xs:string">
                 <xs:annotation>

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/modules/platforms/dotnet/Apache.Ignite.Core/Impl/SwapSpace/SwapSpaceSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/SwapSpace/SwapSpaceSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/SwapSpace/SwapSpaceSerializer.cs
new file mode 100644
index 0000000..7fdbaad
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/SwapSpace/SwapSpaceSerializer.cs
@@ -0,0 +1,99 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Impl.SwapSpace
+{
+    using System;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.SwapSpace;
+    using Apache.Ignite.Core.SwapSpace.File;
+
+    /// <summary>
+    /// SwapSpace config serializer.
+    /// </summary>
+    internal static class SwapSpaceSerializer
+    {
+        /// <summary>
+        /// SwapSpace type.
+        /// </summary>
+        private enum Type : byte
+        {
+            None = 0,
+            File = 1
+        }
+
+        /// <summary>
+        /// Writes the configuration to writer.
+        /// </summary>
+        public static void Write(IBinaryRawWriter writer, ISwapSpaceSpi spi)
+        {
+            Debug.Assert(writer != null);
+
+            var fileSwap = spi as FileSwapSpaceSpi;
+
+            if (spi == null)
+            {
+                writer.WriteByte((byte) Type.None);
+            }
+            else if (fileSwap != null)
+            {
+                writer.WriteByte((byte) Type.File);
+
+                writer.WriteString(fileSwap.BaseDirectory);
+                writer.WriteFloat(fileSwap.MaximumSparsity);
+                writer.WriteInt(fileSwap.MaximumWriteQueueSize);
+                writer.WriteInt(fileSwap.ReadStripesNumber);
+                writer.WriteInt(fileSwap.WriteBufferSize);
+
+            }
+            else
+            {
+                throw new InvalidOperationException("Unsupported swap space SPI: " + spi.GetType());
+            }
+        }
+
+        /// <summary>
+        /// Reads the configuration from reader.
+        /// </summary>
+        public static ISwapSpaceSpi Read(IBinaryRawReader reader)
+        {
+            Debug.Assert(reader != null);
+
+            var type = (Type) reader.ReadByte();
+
+            switch (type)
+            {
+                case Type.None:
+                    return null;
+
+                case Type.File:
+                    return new FileSwapSpaceSpi
+                    {
+                        BaseDirectory = reader.ReadString(),
+                        MaximumSparsity = reader.ReadFloat(),
+                        MaximumWriteQueueSize = reader.ReadInt(),
+                        ReadStripesNumber = reader.ReadInt(),
+                        WriteBufferSize = reader.ReadInt()
+                    };
+
+                default:
+                    throw new ArgumentOutOfRangeException("Invalid Swap Space SPI type: " + type);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/File/FileSwapSpaceSpi.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/File/FileSwapSpaceSpi.cs b/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/File/FileSwapSpaceSpi.cs
new file mode 100644
index 0000000..2140275
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/File/FileSwapSpaceSpi.cs
@@ -0,0 +1,105 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.SwapSpace.File
+{
+    using System;
+    using System.ComponentModel;
+
+    /// <summary>
+    /// File-based swap space SPI implementation which holds keys in memory and values on disk.
+    /// It is intended for cases when value is bigger than 100 bytes, otherwise it will not 
+    /// have any positive effect.
+    /// </summary>
+    public class FileSwapSpaceSpi : ISwapSpaceSpi
+    {
+        /// <summary>
+        /// Default value for <see cref="MaximumSparsity"/> property.
+        /// </summary>
+        public const float DefaultMaximumSparsity = 0.5f;
+
+        /// <summary>
+        /// Default value for <see cref="WriteBufferSize"/> property.
+        /// </summary>
+        public const int DefaultWriteBufferSize = 64 * 1024;
+
+        /// <summary>
+        /// Default value for <see cref="MaximumWriteQueueSize"/> property.
+        /// </summary>
+        public const int DefaultMaximumWriteQueueSize = 1024 * 1024;
+
+        /// <summary>
+        /// Default value for <see cref="ReadStripesNumber"/> property.
+        /// </summary>
+        public static readonly int DefaultReadStripesNumber = Environment.ProcessorCount;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="FileSwapSpaceSpi"/> class.
+        /// </summary>
+        public FileSwapSpaceSpi()
+        {
+            MaximumSparsity = DefaultMaximumSparsity;
+            MaximumWriteQueueSize = DefaultMaximumWriteQueueSize;
+            ReadStripesNumber = DefaultReadStripesNumber;
+            WriteBufferSize = DefaultWriteBufferSize;
+        }
+
+        /// <summary>
+        /// Gets or sets the base directory.
+        /// </summary>
+        public string BaseDirectory { get; set; }
+
+        /// <summary>
+        /// Gets or sets the maximum sparsity. This property defines maximum acceptable
+        /// wasted file space to whole file size ratio.
+        /// When this ratio becomes higher than specified number compacting thread starts working.
+        /// </summary>
+        /// <value>
+        /// The maximum sparsity. Must be between 0 and 1.
+        /// </value>
+        [DefaultValue(DefaultMaximumSparsity)]
+        public float MaximumSparsity { get; set; }
+
+        /// <summary>
+        /// Gets or sets the maximum size of the write queue in bytes. If there are more values are waiting
+        /// to be written to disk then specified size, SPI will block on write operation.
+        /// </summary>
+        /// <value>
+        /// The maximum size of the write queue, in bytes.
+        /// </value>
+        [DefaultValue(DefaultMaximumWriteQueueSize)]
+        public int MaximumWriteQueueSize { get; set; }
+
+        /// <summary>
+        /// Gets or sets the read stripes number. Defines number of file channels to be used concurrently. 
+        /// Default is <see cref="Environment.ProcessorCount"/>.
+        /// </summary>
+        /// <value>
+        /// Number of read stripes.
+        /// </value>
+        public int ReadStripesNumber { get; set; }
+
+        /// <summary>
+        /// Gets or sets the size of the write buffer, in bytes. Write to disk occurs only when this buffer is full.
+        /// </summary>
+        /// <value>
+        /// The size of the write buffer, in bytes.
+        /// </value>
+        [DefaultValue(DefaultWriteBufferSize)]
+        public int WriteBufferSize { get; set; }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d69e26dd/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/ISwapSpaceSpi.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/ISwapSpaceSpi.cs b/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/ISwapSpaceSpi.cs
new file mode 100644
index 0000000..a348b7b
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/SwapSpace/ISwapSpaceSpi.cs
@@ -0,0 +1,35 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.SwapSpace
+{
+    using System.Diagnostics.CodeAnalysis;
+    using Apache.Ignite.Core.SwapSpace.File;
+
+    /// <summary>
+    /// Provides a mechanism in grid for storing data on disk. Ignite cache uses swap space to overflow 
+    /// data to disk if it cannot fit in memory. 
+    /// Logically storage is organized into independent 'spaces' in which data is stored.
+    /// <para />
+    /// Only predefined implementation is supported now: <see cref="FileSwapSpaceSpi"/>.
+    /// </summary>
+    [SuppressMessage("Microsoft.Design", "CA1040:AvoidEmptyInterfaces")]
+    public interface ISwapSpaceSpi
+    {
+        // No-op.
+    }
+}


[19/50] [abbrv] ignite git commit: IGNITE-4028 Get rid of OP_META in PlatformAbstractTarget

Posted by vo...@apache.org.
IGNITE-4028 Get rid of OP_META in PlatformAbstractTarget

This closes #1192


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

Branch: refs/heads/ignite-2693
Commit: ead15193899d08f41491166003cabed0560f0c59
Parents: 3485939
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Nov 7 10:49:03 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Nov 7 10:49:03 2016 +0300

----------------------------------------------------------------------
 .../platform/PlatformAbstractTarget.java        |  11 +-
 .../platform/PlatformNoopProcessor.java         |   5 +
 .../processors/platform/PlatformProcessor.java  |   7 +
 .../platform/PlatformProcessorImpl.java         |   6 +
 .../binary/PlatformBinaryProcessor.java         |  96 ++++++++++++
 .../platform/cluster/PlatformClusterGroup.java  |  31 ----
 .../ignite/impl/binary/binary_type_manager.h    |  14 +-
 .../impl/binary/binary_type_updater_impl.h      |  12 +-
 .../include/ignite/impl/ignite_environment.h    |  49 ++++--
 modules/platforms/cpp/core/src/ignition.cpp     |   8 +-
 .../impl/binary/binary_type_updater_impl.cpp    |  13 +-
 .../cpp/core/src/impl/ignite_environment.cpp    |  45 ++++--
 .../core/src/impl/interop/interop_target.cpp    |   4 +-
 .../cpp/jni/include/ignite/jni/exports.h        |   1 +
 .../platforms/cpp/jni/include/ignite/jni/java.h |   2 +
 .../cpp/jni/include/ignite/jni/utils.h          |  91 ++++++++++-
 modules/platforms/cpp/jni/project/vs/module.def |   1 +
 modules/platforms/cpp/jni/src/exports.cpp       |   4 +
 modules/platforms/cpp/jni/src/java.cpp          |  13 ++
 .../Apache.Ignite.Core.csproj                   |   2 +-
 .../Apache.Ignite.Core/Impl/Binary/Binary.cs    |   2 +-
 .../Impl/Binary/BinaryObjectSchemaSerializer.cs |   2 +-
 .../Impl/Binary/BinaryProcessor.cs              | 156 +++++++++++++++++++
 .../Impl/Binary/BinaryReader.cs                 |   2 +-
 .../Impl/Binary/Marshaller.cs                   |  11 +-
 .../Impl/Cluster/ClusterGroupImpl.cs            |  72 ++-------
 .../Impl/Cluster/IClusterGroupEx.cs             |  35 -----
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  29 ++--
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |   9 +-
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |  64 --------
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |   3 +
 .../Impl/Unmanaged/UnmanagedUtils.cs            |   7 +
 32 files changed, 526 insertions(+), 281 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
index cba0031..2df86ac 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
@@ -43,9 +43,6 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
     /** Constant: ERROR. */
     protected static final int ERROR = -1;
 
-    /** */
-    private static final int OP_META = -1;
-
     /** Context. */
     protected final PlatformContext platformCtx;
 
@@ -78,13 +75,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
         try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
             BinaryRawReaderEx reader = platformCtx.reader(mem);
 
-            if (type == OP_META) {
-                platformCtx.processMetadata(reader);
-
-                return TRUE;
-            }
-            else
-                return processInStreamOutLong(type, reader, mem);
+            return processInStreamOutLong(type, reader, mem);
         }
         catch (Exception e) {
             throw convertException(e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
index a7b7a8d..fd357ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
@@ -185,4 +185,9 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
     @Override public void loggerLog(int level, String message, String category, String errorInfo) {
         // No-op.
     }
+
+    /** {@inheritDoc} */
+    @Override public PlatformTarget binaryProcessor() {
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
index 1d9d3cd..f01175e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
@@ -271,4 +271,11 @@ public interface PlatformProcessor extends GridProcessor {
      * @param errorInfo Error info.
      */
     public void loggerLog(int level, String message, String category, String errorInfo);
+
+    /**
+     * Gets the binary processor.
+     *
+     * @return Binary processor.
+     */
+    public PlatformTarget binaryProcessor();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
index d875c7e..f775987 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
@@ -35,6 +35,7 @@ import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl;
 import org.apache.ignite.internal.processors.datastructures.GridCacheAtomicLongImpl;
+import org.apache.ignite.internal.processors.platform.binary.PlatformBinaryProcessor;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCache;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheExtension;
 import org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinity;
@@ -502,6 +503,11 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
         }
     }
 
+    /** {@inheritDoc} */
+    @Override public PlatformTarget binaryProcessor() {
+        return new PlatformBinaryProcessor(platformCtx);
+    }
+
     /**
      * Gets the near cache config.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java
new file mode 100644
index 0000000..1bb577e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java
@@ -0,0 +1,96 @@
+/*
+ * 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.platform.binary;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.binary.BinaryRawReaderEx;
+import org.apache.ignite.internal.binary.BinaryRawWriterEx;
+import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
+import org.apache.ignite.internal.processors.platform.PlatformContext;
+
+/**
+ * Platform binary processor.
+ */
+public class PlatformBinaryProcessor extends PlatformAbstractTarget {
+    /** */
+    private static final int OP_GET_META = 1;
+
+    /** */
+    private static final int OP_GET_ALL_META = 2;
+
+    /** */
+    private static final int OP_PUT_META = 3;
+
+    /** */
+    private static final int OP_GET_SCHEMA = 4;
+
+    /**
+     * Constructor.
+     *
+     * @param platformCtx Context.
+     */
+    public PlatformBinaryProcessor(PlatformContext platformCtx) {
+        super(platformCtx);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+        if (type == OP_PUT_META) {
+            platformCtx.processMetadata(reader);
+
+            return TRUE;
+        }
+
+        return super.processInStreamOutLong(type, reader);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+        if (type == OP_GET_ALL_META)
+            platformCtx.writeAllMetadata(writer);
+        else
+            super.processOutStream(type, writer);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader,
+        BinaryRawWriterEx writer) throws IgniteCheckedException {
+        switch (type) {
+            case OP_GET_META: {
+                int typeId = reader.readInt();
+
+                platformCtx.writeMetadata(writer, typeId);
+
+                break;
+            }
+
+            case OP_GET_SCHEMA: {
+                int typeId = reader.readInt();
+                int schemaId = reader.readInt();
+
+                platformCtx.writeSchema(writer, typeId, schemaId);
+
+                break;
+            }
+
+            default:
+                super.processInStreamOutStream(type, reader, writer);
+                break;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
index dde98f5..724eea0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
@@ -38,9 +38,6 @@ import org.jetbrains.annotations.Nullable;
 @SuppressWarnings({"UnusedDeclaration"})
 public class PlatformClusterGroup extends PlatformAbstractTarget {
     /** */
-    private static final int OP_ALL_METADATA = 1;
-
-    /** */
     private static final int OP_FOR_ATTRIBUTE = 2;
 
     /** */
@@ -59,9 +56,6 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
     private static final int OP_FOR_NODE_IDS = 7;
 
     /** */
-    private static final int OP_METADATA = 8;
-
-    /** */
     private static final int OP_METRICS = 9;
 
     /** */
@@ -80,9 +74,6 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
     private static final int OP_TOPOLOGY = 14;
 
     /** */
-    private static final int OP_SCHEMA = 15;
-
-    /** */
     private static final int OP_FOR_OTHERS = 16;
 
     /** */
@@ -130,11 +121,6 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
 
                 break;
 
-            case OP_ALL_METADATA:
-                platformCtx.writeAllMetadata(writer);
-
-                break;
-
             default:
                 super.processOutStream(type, writer);
         }
@@ -201,14 +187,6 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
                 break;
             }
 
-            case OP_METADATA: {
-                int typeId = reader.readInt();
-
-                platformCtx.writeMetadata(writer, typeId);
-
-                break;
-            }
-
             case OP_TOPOLOGY: {
                 long topVer = reader.readLong();
 
@@ -217,15 +195,6 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
                 break;
             }
 
-            case OP_SCHEMA: {
-                int typeId = reader.readInt();
-                int schemaId = reader.readInt();
-
-                platformCtx.writeSchema(writer, typeId, schemaId);
-
-                break;
-            }
-
             default:
                 super.processInStreamOutStream(type, reader, writer);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_manager.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_manager.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_manager.h
index 6019a2c..8aca1a0 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_manager.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_type_manager.h
@@ -65,7 +65,7 @@ namespace ignite
                 /**
                  * Get current type manager version.
                  *
-                 * @param Version.
+                 * @return Version.
                  */
                 int32_t GetVersion();
 
@@ -80,7 +80,7 @@ namespace ignite
                 /**
                  * Process pending updates.
                  *
-                 * @param updated Updater.
+                 * @param updater Updater.
                  * @param err Error.
                  * @return In case of success.
                  */
@@ -89,18 +89,18 @@ namespace ignite
             private:
                 /** Current snapshots. */
                 ignite::common::concurrent::SharedPointer<std::map<int32_t, SPSnap>> snapshots;
-                
+
                 /** Pending snapshots. */
-                std::vector<SPSnap>* pending;                                          
+                std::vector<SPSnap>* pending;
 
                 /** Critical section. */
                 ignite::common::concurrent::CriticalSection* cs;
 
                 /** Version of pending changes. */
-                int32_t pendingVer;                                                    
-                
+                int32_t pendingVer;
+
                 /** Latest version. */
-                int32_t ver;          
+                int32_t ver;
 
                 IGNITE_NO_COPY_ASSIGNMENT(BinaryTypeManager);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/core/include/ignite/impl/binary/binary_type_updater_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_type_updater_impl.h b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_type_updater_impl.h
index 4edad4f..bd21751 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/binary/binary_type_updater_impl.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/binary/binary_type_updater_impl.h
@@ -24,7 +24,7 @@
 #include "ignite/impl/binary/binary_type_updater.h"
 
 namespace ignite
-{    
+{
     namespace impl
     {
         namespace binary
@@ -41,7 +41,7 @@ namespace ignite
                  * @param env Environment.
                  * @param javaRef Reference to Java object which is able to process type request.
                  */
-                BinaryTypeUpdaterImpl(ignite::common::concurrent::SharedPointer<IgniteEnvironment> env, jobject javaRef);
+                BinaryTypeUpdaterImpl(IgniteEnvironment& env, jobject javaRef);
 
                 /**
                  * Destructor.
@@ -51,15 +51,15 @@ namespace ignite
                 bool Update(Snap* snapshot, IgniteError* err);
             private:
                 /** Environment. */
-                ignite::common::concurrent::SharedPointer<IgniteEnvironment> env;
-                
+                IgniteEnvironment& env;
+
                 /** Handle to Java object. */
-                jobject javaRef;                 
+                jobject javaRef;
 
                 IGNITE_NO_COPY_ASSIGNMENT(BinaryTypeUpdaterImpl)
             };
         }
-    }    
+    }
 }
 
 #endif //_IGNITE_IMPL_BINARY_BINARY_TYPE_UPDATER_IMPL
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h b/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
index d2486bb..fb6f657 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
@@ -23,6 +23,7 @@
 
 #include "ignite/impl/interop/interop_memory.h"
 #include "ignite/impl/binary/binary_type_manager.h"
+#include "ignite/jni/utils.h"
 
 namespace ignite 
 {
@@ -52,24 +53,29 @@ namespace ignite
             /**
              * Populate callback handlers.
              *
-             * @param Target (current env wrapped into a shared pointer).
+             * @param target (current env wrapped into a shared pointer).
              * @return JNI handlers.
              */
-            ignite::jni::java::JniHandlers GetJniHandlers(ignite::common::concurrent::SharedPointer<IgniteEnvironment>* target);
+            jni::java::JniHandlers GetJniHandlers(common::concurrent::SharedPointer<IgniteEnvironment>* target);
 
             /**
-             * Perform initialization on successful start.
+             * Set context.
              *
              * @param ctx Context.
              */
-            void Initialize(ignite::common::concurrent::SharedPointer<ignite::jni::java::JniContext> ctx);
+            void SetContext(common::concurrent::SharedPointer<jni::java::JniContext> ctx);
+
+            /**
+             * Perform initialization on successful start.
+             */
+            void Initialize();
 
             /**
              * Start callback.
              *
              * @param memPtr Memory pointer.
              */
-            void OnStartCallback(long long memPtr);
+            void OnStartCallback(long long memPtr, jobject proc);
 
             /**
              * Get name of Ignite instance.
@@ -83,14 +89,14 @@ namespace ignite
              *
              * @return Context.
              */
-            ignite::jni::java::JniContext* Context();
+            jni::java::JniContext* Context();
 
             /**
              * Get memory for interop operations.
              *
              * @return Memory.
              */
-            ignite::common::concurrent::SharedPointer<interop::InteropMemory> AllocateMemory();
+            common::concurrent::SharedPointer<interop::InteropMemory> AllocateMemory();
 
             /**
              * Get memory chunk for interop operations with desired capacity.
@@ -98,7 +104,7 @@ namespace ignite
              * @param cap Capacity.
              * @return Memory.
              */
-            ignite::common::concurrent::SharedPointer<interop::InteropMemory> AllocateMemory(int32_t cap);
+            common::concurrent::SharedPointer<interop::InteropMemory> AllocateMemory(int32_t cap);
 
             /**
              * Get memory chunk located at the given pointer.
@@ -106,27 +112,46 @@ namespace ignite
              * @param memPtr Memory pointer.
              * @retrun Memory.
              */
-            ignite::common::concurrent::SharedPointer<interop::InteropMemory> GetMemory(int64_t memPtr);
+            common::concurrent::SharedPointer<interop::InteropMemory> GetMemory(int64_t memPtr);
 
             /**
              * Get type manager.
              *
-             * @param Type manager.
+             * @return Type manager.
              */
             binary::BinaryTypeManager* GetTypeManager();
+
+            /**
+             * Get type updater.
+             *
+             * @return Type updater.
+             */
+            binary::BinaryTypeUpdater* GetTypeUpdater();
+
+            /**
+             * Notify processor that Ignite instance has started.
+             */
+            void ProcessorReleaseStart();
+
         private:
             /** Context to access Java. */
-            ignite::common::concurrent::SharedPointer<ignite::jni::java::JniContext> ctx;
+            common::concurrent::SharedPointer<jni::java::JniContext> ctx;
 
             /** Startup latch. */
-            ignite::common::concurrent::SingleLatch* latch;
+            common::concurrent::SingleLatch* latch;
 
             /** Ignite name. */
             char* name;
 
+            /** Processor instance. */
+            jni::JavaGlobalRef proc;
+
             /** Type manager. */
             binary::BinaryTypeManager* metaMgr;
 
+            /** Type updater. */
+            binary::BinaryTypeUpdater* metaUpdater;
+
             IGNITE_NO_COPY_ASSIGNMENT(IgniteEnvironment);
         };
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/core/src/ignition.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/ignition.cpp b/modules/platforms/cpp/core/src/ignition.cpp
index 83adb4c..72fbf7a 100644
--- a/modules/platforms/cpp/core/src/ignition.cpp
+++ b/modules/platforms/cpp/core/src/ignition.cpp
@@ -230,10 +230,12 @@ namespace ignite
                 if (!ctx.Get())
                 {
                     IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err);
-                    
+
                     failed = true;
                 }
 
+                env.Get()->SetContext(ctx);
+
                 // 5. Start Ignite.
                 if (!failed)
                 {
@@ -263,7 +265,7 @@ namespace ignite
                     }
                     else {
                         // 6. Ignite is started at this point.
-                        env.Get()->Initialize(ctx);
+                        env.Get()->Initialize();
 
                         started = true;
                     }
@@ -288,6 +290,8 @@ namespace ignite
         }
         else 
         {
+            env.Get()->ProcessorReleaseStart();
+
             IgniteImpl* impl = new IgniteImpl(env, javaRef);
 
             return Ignite(impl);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/core/src/impl/binary/binary_type_updater_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/binary/binary_type_updater_impl.cpp b/modules/platforms/cpp/core/src/impl/binary/binary_type_updater_impl.cpp
index e6375a6..ff6df9e 100644
--- a/modules/platforms/cpp/core/src/impl/binary/binary_type_updater_impl.cpp
+++ b/modules/platforms/cpp/core/src/impl/binary/binary_type_updater_impl.cpp
@@ -34,24 +34,25 @@ namespace ignite
         namespace binary
         {
             /** Operation: metadata update. */
-            const int32_t OP_METADATA = -1;
+            const int32_t OP_PUT_META = 3;
 
-            BinaryTypeUpdaterImpl::BinaryTypeUpdaterImpl(SharedPointer<IgniteEnvironment> env,
-                jobject javaRef) :  env(env), javaRef(javaRef)
+            BinaryTypeUpdaterImpl::BinaryTypeUpdaterImpl(IgniteEnvironment& env, jobject javaRef) :
+                env(env),
+                javaRef(javaRef)
             {
                 // No-op.
             }
 
             BinaryTypeUpdaterImpl::~BinaryTypeUpdaterImpl()
             {
-                // No-op.
+                JniContext::Release(javaRef);
             }
 
             bool BinaryTypeUpdaterImpl::Update(Snap* snap, IgniteError* err)
             {
                 JniErrorInfo jniErr;
 
-                SharedPointer<InteropMemory> mem = env.Get()->AllocateMemory();
+                SharedPointer<InteropMemory> mem = env.AllocateMemory();
 
                 InteropOutputStream out(mem.Get());
                 BinaryWriterImpl writer(&out, NULL);
@@ -85,7 +86,7 @@ namespace ignite
 
                 out.Synchronize();
 
-                long long res = env.Get()->Context()->TargetInStreamOutLong(javaRef, OP_METADATA, mem.Get()->PointerLong(), &jniErr);
+                long long res = env.Context()->TargetInStreamOutLong(javaRef, OP_PUT_META, mem.Get()->PointerLong(), &jniErr);
 
                 IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/ignite_environment.cpp b/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
index ab50866..32c855b 100644
--- a/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
+++ b/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
@@ -19,6 +19,7 @@
 #include "ignite/impl/binary/binary_reader_impl.h"
 #include "ignite/impl/ignite_environment.h"
 #include "ignite/binary/binary.h"
+#include "ignite/impl/binary/binary_type_updater_impl.h"
 
 using namespace ignite::common::concurrent;
 using namespace ignite::jni::java;
@@ -34,14 +35,14 @@ namespace ignite
          * OnStart callback.
          *
          * @param target Target environment.
-         * @param proc Processor instance (not used for now).
+         * @param proc Processor instance.
          * @param memPtr Memory pointer.
          */
         void IGNITE_CALL OnStart(void* target, void* proc, long long memPtr)
         {
             SharedPointer<IgniteEnvironment>* ptr = static_cast<SharedPointer<IgniteEnvironment>*>(target);
 
-            ptr->Get()->OnStartCallback(memPtr);
+            ptr->Get()->OnStartCallback(memPtr, reinterpret_cast<jobject>(proc));
         }
 
         /**
@@ -72,8 +73,8 @@ namespace ignite
             mem.Get()->Reallocate(cap);
         }
 
-        IgniteEnvironment::IgniteEnvironment() : ctx(SharedPointer<JniContext>()), latch(new SingleLatch), name(NULL),
-            metaMgr(new BinaryTypeManager())
+        IgniteEnvironment::IgniteEnvironment() : ctx(SharedPointer<JniContext>()), latch(new SingleLatch), name(0),
+            proc(), metaMgr(new BinaryTypeManager()), metaUpdater(0)
         {
             // No-op.
         }
@@ -81,11 +82,9 @@ namespace ignite
         IgniteEnvironment::~IgniteEnvironment()
         {
             delete latch;
-
-            if (name)
-                delete name;
-
+            delete name;
             delete metaMgr;
+            delete metaUpdater;
         }
 
         JniHandlers IgniteEnvironment::GetJniHandlers(SharedPointer<IgniteEnvironment>* target)
@@ -99,16 +98,23 @@ namespace ignite
 
             hnds.memRealloc = MemoryReallocate;
 
-            hnds.error = NULL;
+            hnds.error = 0;
 
             return hnds;
         }
 
-        void IgniteEnvironment::Initialize(SharedPointer<JniContext> ctx)
+        void IgniteEnvironment::SetContext(SharedPointer<JniContext> ctx)
         {
             this->ctx = ctx;
+        }
 
+        void IgniteEnvironment::Initialize()
+        {
             latch->CountDown();
+
+            jobject binaryProc = Context()->ProcessorBinaryProcessor(proc.Get());
+
+            metaUpdater = new BinaryTypeUpdaterImpl(*this, binaryProc);
         }
 
         const char* IgniteEnvironment::InstanceName() const
@@ -160,14 +166,27 @@ namespace ignite
             return metaMgr;
         }
 
-        void IgniteEnvironment::OnStartCallback(long long memPtr)
+        BinaryTypeUpdater* IgniteEnvironment::GetTypeUpdater()
         {
+            return metaUpdater;
+        }
+
+        void IgniteEnvironment::ProcessorReleaseStart()
+        {
+            if (proc.Get())
+                ctx.Get()->ProcessorReleaseStart(proc.Get());
+        }
+
+        void IgniteEnvironment::OnStartCallback(long long memPtr, jobject proc)
+        {
+            this->proc = jni::JavaGlobalRef(*ctx.Get(), proc);
+
             InteropExternalMemory mem(reinterpret_cast<int8_t*>(memPtr));
             InteropInputStream stream(&mem);
 
             BinaryReaderImpl reader(&stream);
 
-            int32_t nameLen = reader.ReadString(NULL, 0);
+            int32_t nameLen = reader.ReadString(0, 0);
 
             if (nameLen >= 0)
             {
@@ -175,7 +194,7 @@ namespace ignite
                 reader.ReadString(name, nameLen + 1);
             }
             else
-                name = NULL;
+                name = 0;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp b/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp
index 196c3f6..4992ccb 100644
--- a/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp
+++ b/modules/platforms/cpp/core/src/impl/interop/interop_target.cpp
@@ -72,9 +72,7 @@ namespace ignite
 
                 if (metaMgr->IsUpdatedSince(metaVer))
                 {
-                    BinaryTypeUpdaterImpl metaUpdater(env, javaRef);
-
-                    if (!metaMgr->ProcessPendingUpdates(&metaUpdater, err))
+                    if (!metaMgr->ProcessPendingUpdates(env.Get()->GetTypeUpdater(), err))
                         return 0;
                 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/jni/include/ignite/jni/exports.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/include/ignite/jni/exports.h b/modules/platforms/cpp/jni/include/ignite/jni/exports.h
index 3a98eda..3052435 100644
--- a/modules/platforms/cpp/jni/include/ignite/jni/exports.h
+++ b/modules/platforms/cpp/jni/include/ignite/jni/exports.h
@@ -56,6 +56,7 @@ extern "C" {
     void IGNITE_CALL IgniteProcessorGetCacheNames(gcj::JniContext* ctx, void* obj, long long memPtr);
     bool IGNITE_CALL IgniteProcessorLoggerIsLevelEnabled(gcj::JniContext* ctx, void* obj, int level);
     void IGNITE_CALL IgniteProcessorLoggerLog(gcj::JniContext* ctx, void* obj, int level, char* message, char* category, char* errorInfo);
+    void* IGNITE_CALL IgniteProcessorBinaryProcessor(gcj::JniContext* ctx, void* obj);
 
     long long IGNITE_CALL IgniteTargetInLongOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr);
     long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/jni/include/ignite/jni/java.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/include/ignite/jni/java.h b/modules/platforms/cpp/jni/include/ignite/jni/java.h
index 9dddd8c..07df001 100644
--- a/modules/platforms/cpp/jni/include/ignite/jni/java.h
+++ b/modules/platforms/cpp/jni/include/ignite/jni/java.h
@@ -269,6 +269,7 @@ namespace ignite
                 jmethodID m_PlatformProcessor_atomicReference;
                 jmethodID m_PlatformProcessor_loggerIsLevelEnabled;
                 jmethodID m_PlatformProcessor_loggerLog;
+                jmethodID m_PlatformProcessor_binaryProcessor;
 
                 jclass c_PlatformTarget;
                 jmethodID m_PlatformTarget_inLongOutLong;
@@ -449,6 +450,7 @@ namespace ignite
 				void ProcessorGetCacheNames(jobject obj, long long memPtr);
 				bool ProcessorLoggerIsLevelEnabled(jobject obj, int level);
 				void ProcessorLoggerLog(jobject obj, int level, char* message, char* category, char* errorInfo);
+                jobject ProcessorBinaryProcessor(jobject obj);
 
                 long long TargetInLongOutLong(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL);
                 long long TargetInStreamOutLong(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/jni/include/ignite/jni/utils.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/include/ignite/jni/utils.h b/modules/platforms/cpp/jni/include/ignite/jni/utils.h
index ee40248..2b22cf9 100644
--- a/modules/platforms/cpp/jni/include/ignite/jni/utils.h
+++ b/modules/platforms/cpp/jni/include/ignite/jni/utils.h
@@ -19,6 +19,7 @@
 
 #include <string>
 
+#include <ignite/jni/java.h>
 #include <ignite/common/common.h>
 
 namespace ignite
@@ -35,7 +36,7 @@ namespace ignite
              * Destructor.
              */
             ~AttachHelper();
-			
+
             /**
              * Callback invoked on successful thread attach ot JVM.
              */
@@ -43,6 +44,94 @@ namespace ignite
         };
 
         /**
+        * Represents global reference to Java object.
+        */
+        class IGNITE_IMPORT_EXPORT JavaGlobalRef
+        {
+        public:
+            /**
+            * Default constructor
+            */
+            JavaGlobalRef() :
+                ctx(0),
+                obj(0)
+            {
+                // No-op.
+            }
+
+            /**
+            * Constructor
+            *
+            * @param ctx JNI context.
+            * @param obj Java object.
+            */
+            JavaGlobalRef(java::JniContext& ctx, jobject obj) :
+                ctx(&ctx),
+                obj(ctx.Acquire(obj))
+            {
+                // No-op.
+            }
+
+            /**
+            * Copy constructor
+            *
+            * @param other Other instance.
+            */
+            JavaGlobalRef(const JavaGlobalRef& other) :
+                ctx(other.ctx),
+                obj(ctx->Acquire(other.obj))
+            {
+                // No-op.
+            }
+
+            /**
+            * Assignment operator.
+            *
+            * @param other Other instance.
+            * @return *this.
+            */
+            JavaGlobalRef& operator=(const JavaGlobalRef& other)
+            {
+                if (this != &other)
+                {
+                    if (ctx)
+                        ctx->Release(obj);
+
+                    ctx = other.ctx;
+                    obj = ctx->Acquire(other.obj);
+                }
+
+                return *this;
+            }
+
+            /**
+            * Destructor.
+            */
+            ~JavaGlobalRef()
+            {
+                if (ctx)
+                    ctx->Release(obj);
+            }
+
+            /**
+            * Get object.
+            *
+            * @return Object.
+            */
+            jobject Get()
+            {
+                return obj;
+            }
+
+        private:
+            /** Context. */
+            java::JniContext* ctx;
+
+            /** Object. */
+            jobject obj;
+        };
+
+        /**
          * Attempts to find JVM library to load it into the process later.
          * First search is performed using the passed path argument (is not NULL).
          * Then JRE_HOME is evaluated. Last, JAVA_HOME is evaluated.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/jni/project/vs/module.def
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/project/vs/module.def b/modules/platforms/cpp/jni/project/vs/module.def
index 258e80c..e58ac3b 100644
--- a/modules/platforms/cpp/jni/project/vs/module.def
+++ b/modules/platforms/cpp/jni/project/vs/module.def
@@ -51,3 +51,4 @@ IgniteSetConsoleHandler @135
 IgniteRemoveConsoleHandler @136
 IgniteProcessorLoggerIsLevelEnabled @137
 IgniteProcessorLoggerLog @138
+IgniteProcessorBinaryProcessor @139
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/jni/src/exports.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/src/exports.cpp b/modules/platforms/cpp/jni/src/exports.cpp
index e87cbd3..dde98fb 100644
--- a/modules/platforms/cpp/jni/src/exports.cpp
+++ b/modules/platforms/cpp/jni/src/exports.cpp
@@ -150,6 +150,10 @@ extern "C" {
         ctx->ProcessorLoggerLog(static_cast<jobject>(obj), level, message, category, errorInfo);
     }
 
+    void* IGNITE_CALL IgniteProcessorBinaryProcessor(gcj::JniContext* ctx, void* obj) {
+        return ctx->ProcessorBinaryProcessor(static_cast<jobject>(obj));
+    }
+
     long long IGNITE_CALL IgniteTargetInStreamOutLong(gcj::JniContext* ctx, void* obj, int opType, long long memPtr) {
         return ctx->TargetInStreamOutLong(static_cast<jobject>(obj), opType, memPtr);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/cpp/jni/src/java.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/src/java.cpp b/modules/platforms/cpp/jni/src/java.cpp
index e2c9bf7..2d3cf72 100644
--- a/modules/platforms/cpp/jni/src/java.cpp
+++ b/modules/platforms/cpp/jni/src/java.cpp
@@ -218,6 +218,7 @@ namespace ignite
             JniMethod M_PLATFORM_PROCESSOR_GET_CACHE_NAMES = JniMethod("getCacheNames", "(J)V", false);
             JniMethod M_PLATFORM_PROCESSOR_LOGGER_IS_LEVEL_ENABLED = JniMethod("loggerIsLevelEnabled", "(I)Z", false);
             JniMethod M_PLATFORM_PROCESSOR_LOGGER_LOG = JniMethod("loggerLog", "(ILjava/lang/String;Ljava/lang/String;Ljava/lang/String;)V", false);
+            JniMethod M_PLATFORM_PROCESSOR_BINARY_PROCESSOR = JniMethod("binaryProcessor", "()Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
 
             const char* C_PLATFORM_TARGET = "org/apache/ignite/internal/processors/platform/PlatformTarget";
             JniMethod M_PLATFORM_TARGET_IN_LONG_OUT_LONG = JniMethod("inLongOutLong", "(IJ)J", false);
@@ -534,6 +535,7 @@ namespace ignite
 				m_PlatformProcessor_getCacheNames = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_GET_CACHE_NAMES);
 				m_PlatformProcessor_loggerIsLevelEnabled = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_LOGGER_IS_LEVEL_ENABLED);
 				m_PlatformProcessor_loggerLog = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_LOGGER_LOG);
+				m_PlatformProcessor_binaryProcessor = FindMethod(env, c_PlatformProcessor, M_PLATFORM_PROCESSOR_BINARY_PROCESSOR);
 
                 c_PlatformTarget = FindClass(env, C_PLATFORM_TARGET);
                 m_PlatformTarget_inLongOutLong = FindMethod(env, c_PlatformTarget, M_PLATFORM_TARGET_IN_LONG_OUT_LONG);
@@ -1339,6 +1341,17 @@ namespace ignite
                 ExceptionCheck(env);
             }
 
+            jobject JniContext::ProcessorBinaryProcessor(jobject obj)
+            {
+                JNIEnv* env = Attach();
+
+                jobject res = env->CallObjectMethod(obj, jvm->GetMembers().m_PlatformProcessor_binaryProcessor);
+
+                ExceptionCheck(env);
+
+                return LocalToGlobal(env, res);
+            }
+
             long long JniContext::TargetInStreamOutLong(jobject obj, int opType, long long memPtr, JniErrorInfo* err) {
                 JNIEnv* env = Attach();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 8e8f8ca..2973bb3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -91,6 +91,7 @@
     <Compile Include="Discovery\Tcp\Multicast\Package-Info.cs" />
     <Compile Include="Discovery\Tcp\Package-Info.cs" />
     <Compile Include="Discovery\Tcp\Static\Package-Info.cs" />
+    <Compile Include="Impl\Binary\BinaryProcessor.cs" />
     <Compile Include="Impl\Binary\BinaryReflectiveSerializerInternal.cs" />
     <Compile Include="Impl\Binary\IBinarySerializerInternal.cs" />
     <Compile Include="Binary\Package-Info.cs" />
@@ -287,7 +288,6 @@
     <Compile Include="Impl\Cluster\ClusterGroupImpl.cs" />
     <Compile Include="Impl\Cluster\ClusterMetricsImpl.cs" />
     <Compile Include="Impl\Cluster\ClusterNodeImpl.cs" />
-    <Compile Include="Impl\Cluster\IClusterGroupEx.cs" />
     <Compile Include="Impl\Collections\CollectionExtensions.cs" />
     <Compile Include="Impl\Collections\MultiValueDictionary.cs" />
     <Compile Include="Impl\Collections\ReadOnlyCollection.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
index 1deac07..3d55acd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
@@ -120,7 +120,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /** <inheritDoc /> */
         public ICollection<IBinaryType> GetBinaryTypes()
         {
-            return Marshaller.Ignite.ClusterGroup.GetBinaryTypes();
+            return Marshaller.Ignite.BinaryProcessor.GetBinaryTypes();
         }
 
         /** <inheritDoc /> */

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs
index da86c07..f1d2f6a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaSerializer.cs
@@ -250,7 +250,7 @@ namespace Apache.Ignite.Core.Impl.Binary
             if (fieldIds == null)
             {
                 if (marsh.Ignite != null)
-                    fieldIds = marsh.Ignite.ClusterGroup.GetSchema(hdr.TypeId, hdr.SchemaId);
+                    fieldIds = marsh.Ignite.BinaryProcessor.GetSchema(hdr.TypeId, hdr.SchemaId);
 
                 if (fieldIds == null)
                     throw new BinaryObjectException("Cannot find schema for object with compact footer [" +

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessor.cs
new file mode 100644
index 0000000..6935fa2
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessor.cs
@@ -0,0 +1,156 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Impl.Binary
+{
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Impl.Binary.Metadata;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+
+    /// <summary>
+    /// Binary metadata processor.
+    /// </summary>
+    internal class BinaryProcessor : PlatformTarget
+    {
+        /// <summary>
+        /// Op codes.
+        /// </summary>
+        private enum Op
+        {
+            GetMeta = 1,
+            GetAllMeta = 2,
+            PutMeta = 3,
+            GetSchema = 4
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="BinaryProcessor"/> class.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaller.</param>
+        public BinaryProcessor(IUnmanagedTarget target, Marshaller marsh) : base(target, marsh)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Gets metadata for specified type.
+        /// </summary>
+        public IBinaryType GetBinaryType(int typeId)
+        {
+            return DoOutInOp<IBinaryType>((int) Op.GetMeta,
+                writer => writer.WriteInt(typeId),
+                stream =>
+                {
+                    var reader = Marshaller.StartUnmarshal(stream, false);
+
+                    return reader.ReadBoolean() ? new BinaryType(reader) : null;
+                }
+            );
+        }
+
+        /// <summary>
+        /// Gets metadata for all known types.
+        /// </summary>
+        public List<IBinaryType> GetBinaryTypes()
+        {
+            return DoInOp((int) Op.GetAllMeta, s =>
+            {
+                var reader = Marshaller.StartUnmarshal(s);
+
+                var size = reader.ReadInt();
+
+                var res = new List<IBinaryType>(size);
+
+                for (var i = 0; i < size; i++)
+                    res.Add(reader.ReadBoolean() ? new BinaryType(reader) : null);
+
+                return res;
+            });
+        }
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        public int[] GetSchema(int typeId, int schemaId)
+        {
+            return DoOutInOp<int[]>((int) Op.GetSchema, writer =>
+            {
+                writer.WriteInt(typeId);
+                writer.WriteInt(schemaId);
+            });
+        }
+
+        /// <summary>
+        /// Put binary types to Grid.
+        /// </summary>
+        /// <param name="types">Binary types.</param>
+        internal void PutBinaryTypes(ICollection<BinaryType> types)
+        {
+            DoOutOp((int) Op.PutMeta, w =>
+            {
+                w.WriteInt(types.Count);
+
+                foreach (var meta in types)
+                {
+                    w.WriteInt(meta.TypeId);
+                    w.WriteString(meta.TypeName);
+                    w.WriteString(meta.AffinityKeyFieldName);
+
+                    var fields = meta.GetFieldsMap();
+
+                    w.WriteInt(fields.Count);
+
+                    foreach (var field in fields)
+                    {
+                        w.WriteString(field.Key);
+                        w.WriteInt(field.Value);
+                    }
+
+                    w.WriteBoolean(meta.IsEnum);
+
+                    // Send schemas
+                    var desc = meta.Descriptor;
+                    Debug.Assert(desc != null);
+
+                    var count = 0;
+                    var countPos = w.Stream.Position;
+                    w.WriteInt(0); // Reserve for count
+
+                    foreach (var schema in desc.Schema.GetAll())
+                    {
+                        w.WriteInt(schema.Key);
+
+                        var ids = schema.Value;
+                        w.WriteInt(ids.Length);
+
+                        foreach (var id in ids)
+                            w.WriteInt(id);
+
+                        count++;
+                    }
+
+                    w.Stream.WriteInt(countPos, count);
+                }
+            });
+
+            Marshaller.OnBinaryTypesSent(types);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
index f048e97..4c34f73 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
@@ -768,7 +768,7 @@ namespace Apache.Ignite.Core.Impl.Binary
             if (_curHdr.IsCompactFooter)
             {
                 // Get schema from Java
-                var schema = Marshaller.Ignite.ClusterGroup.GetSchema(_curHdr.TypeId, _curHdr.SchemaId);
+                var schema = Marshaller.Ignite.BinaryProcessor.GetSchema(_curHdr.TypeId, _curHdr.SchemaId);
 
                 if (schema == null)
                     throw new BinaryObjectException("Cannot find schema for object with compact footer [" +

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
index 6e63e9a..7acdfaa 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
@@ -161,7 +161,9 @@ namespace Apache.Ignite.Core.Impl.Binary
             var ignite = Ignite;
 
             if (ignite != null && metas != null && metas.Count > 0)
-                ignite.PutBinaryTypes(metas);
+            {
+                ignite.BinaryProcessor.PutBinaryTypes(metas);
+            }
         }
 
         /// <summary>
@@ -270,7 +272,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             if (Ignite != null)
             {
-                IBinaryType meta = Ignite.GetBinaryType(typeId);
+                IBinaryType meta = Ignite.BinaryProcessor.GetBinaryType(typeId);
 
                 if (meta != null)
                     return meta;
@@ -290,7 +292,10 @@ namespace Apache.Ignite.Core.Impl.Binary
             GetBinaryTypeHandler(desc);  // ensure that handler exists
 
             if (Ignite != null)
-                Ignite.PutBinaryTypes(new[] {new BinaryType(desc)});
+            {
+                ICollection<BinaryType> metas = new[] {new BinaryType(desc)};
+                Ignite.BinaryProcessor.PutBinaryTypes(metas);
+            }
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
index 277d61f..6c8779c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterGroupImpl.cs
@@ -29,7 +29,6 @@ namespace Apache.Ignite.Core.Impl.Cluster
     using Apache.Ignite.Core.Compute;
     using Apache.Ignite.Core.Events;
     using Apache.Ignite.Core.Impl.Binary;
-    using Apache.Ignite.Core.Impl.Binary.Metadata;
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Impl.Compute;
     using Apache.Ignite.Core.Impl.Events;
@@ -43,7 +42,7 @@ namespace Apache.Ignite.Core.Impl.Cluster
     /// <summary>
     /// Ignite projection implementation.
     /// </summary>
-    internal class ClusterGroupImpl : PlatformTarget, IClusterGroupEx
+    internal class ClusterGroupImpl : PlatformTarget, IClusterGroup
     {
         /** Attribute: platform. */
         private const string AttrPlatform = "org.apache.ignite.platform";
@@ -55,9 +54,6 @@ namespace Apache.Ignite.Core.Impl.Cluster
         private const int TopVerInit = 0;
 
         /** */
-        private const int OpAllMetadata = 1;
-
-        /** */
         private const int OpForAttribute = 2;
 
         /** */
@@ -76,9 +72,6 @@ namespace Apache.Ignite.Core.Impl.Cluster
         private const int OpForNodeIds = 7;
 
         /** */
-        private const int OpMetadata = 8;
-
-        /** */
         private const int OpMetrics = 9;
 
         /** */
@@ -97,13 +90,10 @@ namespace Apache.Ignite.Core.Impl.Cluster
         private const int OpTopology = 14;
 
         /** */
-        private const int OpSchema = 15;
-
-        /** */
         private const int OpForRemotes = 17;
 
         /** */
-        public const int OpForDaemons = 18;
+        private const int OpForDaemons = 18;
 
         /** */
         private const int OpForRandom = 19;
@@ -115,10 +105,10 @@ namespace Apache.Ignite.Core.Impl.Cluster
         private const int OpForYoungest = 21;
         
         /** */
-        public const int OpResetMetrics = 22;
+        private const int OpResetMetrics = 22;
         
         /** */
-        public const int OpForServers = 23;
+        private const int OpForServers = 23;
         
         /** Initial Ignite instance. */
         private readonly Ignite _ignite;
@@ -508,6 +498,14 @@ namespace Apache.Ignite.Core.Impl.Cluster
         }
 
         /// <summary>
+        /// Resets the metrics.
+        /// </summary>
+        public void ResetMetrics()
+        {
+            DoOutInOp(OpResetMetrics);
+        }
+
+        /// <summary>
         /// Creates new Cluster Group from given native projection.
         /// </summary>
         /// <param name="prj">Native projection.</param>
@@ -553,51 +551,5 @@ namespace Apache.Ignite.Core.Impl.Cluster
 
             return _nodes;
         }
-
-        /** <inheritDoc /> */
-        public IBinaryType GetBinaryType(int typeId)
-        {
-            return DoOutInOp<IBinaryType>(OpMetadata,
-                writer => writer.WriteInt(typeId),
-                stream =>
-                {
-                    var reader = Marshaller.StartUnmarshal(stream, false);
-
-                    return reader.ReadBoolean() ? new BinaryType(reader) : null;
-                }
-            );
-        }
-
-        /// <summary>
-        /// Gets metadata for all known types.
-        /// </summary>
-        public List<IBinaryType> GetBinaryTypes()
-        {
-            return DoInOp(OpAllMetadata, s =>
-            {
-                var reader = Marshaller.StartUnmarshal(s);
-
-                var size = reader.ReadInt();
-
-                var res = new List<IBinaryType>(size);
-
-                for (var i = 0; i < size; i++)
-                    res.Add(reader.ReadBoolean() ? new BinaryType(reader) : null);
-
-                return res;
-            });
-        }
-
-        /// <summary>
-        /// Gets the schema.
-        /// </summary>
-        public int[] GetSchema(int typeId, int schemaId)
-        {
-            return DoOutInOp<int[]>(OpSchema, writer =>
-            {
-                writer.WriteInt(typeId);
-                writer.WriteInt(schemaId);
-            });
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/IClusterGroupEx.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/IClusterGroupEx.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/IClusterGroupEx.cs
deleted file mode 100644
index 59162e6..0000000
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/IClusterGroupEx.cs
+++ /dev/null
@@ -1,35 +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.
- */
-
-namespace Apache.Ignite.Core.Impl.Cluster
-{
-    using Apache.Ignite.Core.Binary;
-    using Apache.Ignite.Core.Cluster;
-
-    /// <summary>
-    /// Extended internal Ignite interface.
-    /// </summary>
-    internal interface IClusterGroupEx : IClusterGroup
-    {
-        /// <summary>
-        /// Gets protable metadata for type.
-        /// </summary>
-        /// <param name="typeId">Type ID.</param>
-        /// <returns>Metadata.</returns>
-        IBinaryType GetBinaryType(int typeId);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
index e8171cb..1747df2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
@@ -33,7 +33,6 @@ namespace Apache.Ignite.Core.Impl
     using Apache.Ignite.Core.DataStructures;
     using Apache.Ignite.Core.Events;
     using Apache.Ignite.Core.Impl.Binary;
-    using Apache.Ignite.Core.Impl.Binary.Metadata;
     using Apache.Ignite.Core.Impl.Cache;
     using Apache.Ignite.Core.Impl.Cluster;
     using Apache.Ignite.Core.Impl.Common;
@@ -52,7 +51,7 @@ namespace Apache.Ignite.Core.Impl
     /// <summary>
     /// Native Ignite wrapper.
     /// </summary>
-    internal class Ignite : IIgnite, IClusterGroupEx, ICluster
+    internal class Ignite : IIgnite, ICluster
     {
         /** */
         private readonly IgniteConfiguration _cfg;
@@ -72,6 +71,9 @@ namespace Apache.Ignite.Core.Impl
         /** Binary. */
         private readonly Binary.Binary _binary;
 
+        /** Binary processor. */
+        private readonly BinaryProcessor _binaryProc;
+
         /** Cached proxy. */
         private readonly IgniteProxy _proxy;
 
@@ -126,6 +128,8 @@ namespace Apache.Ignite.Core.Impl
 
             _binary = new Binary.Binary(marsh);
 
+            _binaryProc = new BinaryProcessor(UU.ProcessorBinaryProcessor(proc), marsh);
+
             _proxy = new IgniteProxy(this);
 
             cbs.Initialize(this);
@@ -517,7 +521,7 @@ namespace Apache.Ignite.Core.Impl
         /** <inheritdoc /> */
         public void ResetMetrics()
         {
-            UU.TargetInLongOutLong(_prj.Target, ClusterGroupImpl.OpResetMetrics, 0);
+            _prj.ResetMetrics();
         }
 
         /** <inheritdoc /> */
@@ -732,26 +736,19 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /// <summary>
-        /// Configuration.
+        /// Gets the binary processor.
         /// </summary>
-        internal IgniteConfiguration Configuration
+        internal BinaryProcessor BinaryProcessor
         {
-            get { return _cfg; }
+            get { return _binaryProc; }
         }
 
         /// <summary>
-        /// Put metadata to Grid.
+        /// Configuration.
         /// </summary>
-        /// <param name="metas">Metadata.</param>
-        internal void PutBinaryTypes(ICollection<BinaryType> metas)
-        {
-            _prj.PutBinaryTypes(metas);
-        }
-
-        /** <inheritDoc /> */
-        public IBinaryType GetBinaryType(int typeId)
+        internal IgniteConfiguration Configuration
         {
-            return _prj.GetBinaryType(typeId);
+            get { return _cfg; }
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
index 8dc63bd..98a2d47 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs
@@ -30,7 +30,6 @@ namespace Apache.Ignite.Core.Impl
     using Apache.Ignite.Core.DataStructures;
     using Apache.Ignite.Core.Events;
     using Apache.Ignite.Core.Impl.Binary;
-    using Apache.Ignite.Core.Impl.Cluster;
     using Apache.Ignite.Core.Log;
     using Apache.Ignite.Core.Lifecycle;
     using Apache.Ignite.Core.Messaging;
@@ -42,7 +41,7 @@ namespace Apache.Ignite.Core.Impl
     /// </summary>
     [Serializable]
     [ExcludeFromCodeCoverage]
-    internal class IgniteProxy : IIgnite, IClusterGroupEx, IBinaryWriteAware, ICluster
+    internal class IgniteProxy : IIgnite, IBinaryWriteAware, ICluster
     {
         /** */
         [NonSerialized]
@@ -455,11 +454,5 @@ namespace Apache.Ignite.Core.Impl
                 return _ignite;
             }
         }
-
-        /** <inheritdoc /> */
-        public IBinaryType GetBinaryType(int typeId)
-        {
-            return _ignite.GetBinaryType(typeId);
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
index fb9d890..bafc759 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
@@ -26,7 +26,6 @@ namespace Apache.Ignite.Core.Impl
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Binary.IO;
-    using Apache.Ignite.Core.Impl.Binary.Metadata;
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Impl.Memory;
     using Apache.Ignite.Core.Impl.Unmanaged;
@@ -50,9 +49,6 @@ namespace Apache.Ignite.Core.Impl
         protected const int Error = -1;
 
         /** */
-        private const int OpMeta = -1;
-
-        /** */
         public const int OpNone = -2;
 
         /** */
@@ -827,66 +823,6 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /// <summary>
-        /// Put binary types to Grid.
-        /// </summary>
-        /// <param name="types">Binary types.</param>
-        internal void PutBinaryTypes(ICollection<BinaryType> types)
-        {
-            DoOutOp(OpMeta, stream =>
-            {
-                BinaryWriter w = _marsh.StartMarshal(stream);
-
-                w.WriteInt(types.Count);
-
-                foreach (var meta in types)
-                {
-                    w.WriteInt(meta.TypeId);
-                    w.WriteString(meta.TypeName);
-                    w.WriteString(meta.AffinityKeyFieldName);
-
-                    IDictionary<string, int> fields = meta.GetFieldsMap();
-
-                    w.WriteInt(fields.Count);
-
-                    foreach (var field in fields)
-                    {
-                        w.WriteString(field.Key);
-                        w.WriteInt(field.Value);
-                    }
-
-                    w.WriteBoolean(meta.IsEnum);
-
-                    // Send schemas
-                    var desc = meta.Descriptor;
-                    Debug.Assert(desc != null);
-
-                    var count = 0;
-                    var countPos = stream.Position;
-                    w.WriteInt(0);  // Reserve for count
-
-                    foreach (var schema in desc.Schema.GetAll())
-                    {
-                        w.WriteInt(schema.Key);
-
-                        var ids = schema.Value;
-                        w.WriteInt(ids.Length);
-
-                        foreach (var id in ids)
-                            w.WriteInt(id);
-
-                        count++;
-                    }
-
-                    stream.WriteInt(countPos, count);
-                }
-
-                _marsh.FinishMarshal(w);
-            });
-
-            _marsh.OnBinaryTypesSent(types);
-        }
-
-        /// <summary>
         /// Unmarshal object using the given stream.
         /// </summary>
         /// <param name="stream">Stream.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
index ac41f25..c746866 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
@@ -123,6 +123,9 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
         [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorLoggerLog")]
         public static extern void ProcessorLoggerLog(void* ctx, void* obj, int level, sbyte* messsage, sbyte* category, sbyte* errorInfo);
 
+        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorBinaryProcessor")]
+        public static extern void* ProcessorBinaryProcessor(void* ctx, void* obj);
+
         [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetInStreamOutLong")]
         public static extern long TargetInStreamOutLong(void* ctx, void* target, int opType, long memPtr);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ead15193/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
index fe1904c..f36c35f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
@@ -402,6 +402,13 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
             }
         }
 
+        internal static IUnmanagedTarget ProcessorBinaryProcessor(IUnmanagedTarget target)
+        {
+            void* res = JNI.ProcessorBinaryProcessor(target.Context, target.Target);
+
+            return target.ChangeTarget(res);
+        }
+
         #endregion
 
         #region NATIVE METHODS: TARGET


[33/50] [abbrv] ignite git commit: IGNITE-1915 .NET: Ignite as Entity Framework Second-Level Cache

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCache.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCache.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCache.cs
new file mode 100644
index 0000000..a7ac2c9
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCache.cs
@@ -0,0 +1,295 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.EntityFramework.Impl
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Data.Entity.Core.Metadata.Edm;
+    using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
+    using System.IO;
+    using System.Linq;
+    using System.Runtime.Serialization.Formatters.Binary;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Cache.Expiry;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Cache;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.Core.Log;
+
+    /// <summary>
+    /// Database query cache.
+    /// </summary>
+    internal class DbCache
+    {
+        /** Extension id.  */
+        private const int ExtensionId = 1;
+
+        /** Invalidate sets extension operation. */
+        private const int OpInvalidateSets = 1;
+
+        /** Put data extension operation. */
+        private const int OpPutItem = 2;
+
+        /** Get data extension operation. */
+        private const int OpGetItem = 3;
+
+        /** Max number of cached expiry caches. */
+        private const int MaxExpiryCaches = 1000;
+
+        /** Main cache: stores SQL -> QueryResult mappings. */
+        private readonly ICache<string, object> _cache;
+
+        /** Entity set version cache. */
+        private readonly ICache<string, long> _metaCache;
+
+        /** Cached caches per (expiry_seconds * 10). */
+        private volatile Dictionary<long, ICache<string, object>> _expiryCaches =
+            new Dictionary<long, ICache<string, object>>();
+
+        /** Sync object. */
+        private readonly object _syncRoot = new object();
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="DbCache" /> class.
+        /// </summary>
+        /// <param name="ignite">The ignite.</param>
+        /// <param name="metaCacheConfiguration">The meta cache configuration.</param>
+        /// <param name="dataCacheConfiguration">The data cache configuration.</param>
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods",
+            Justification = "Validation is present")]
+        public DbCache(IIgnite ignite, CacheConfiguration metaCacheConfiguration, 
+            CacheConfiguration dataCacheConfiguration)
+        {
+            IgniteArgumentCheck.NotNull(ignite, "ignite");
+            IgniteArgumentCheck.NotNull(metaCacheConfiguration, "metaCacheConfiguration");
+            IgniteArgumentCheck.NotNull(dataCacheConfiguration, "metaCacheConfiguration");
+
+            IgniteArgumentCheck.Ensure(metaCacheConfiguration.Name != dataCacheConfiguration.Name, 
+                "dataCacheConfiguration", "Meta and Data cache can't have the same name.");
+
+            _metaCache = ignite.GetOrCreateCache<string, long>(metaCacheConfiguration);
+            _cache = ignite.GetOrCreateCache<string, object>(dataCacheConfiguration);
+
+            var metaCfg = _metaCache.GetConfiguration();
+
+            if (metaCfg.AtomicityMode != CacheAtomicityMode.Transactional)
+                throw new IgniteException("EntityFramework meta cache should be Transactional.");
+
+            if (metaCfg.CacheMode == CacheMode.Partitioned && metaCfg.Backups < 1)
+                ignite.Logger.Warn("EntityFramework meta cache is partitioned and has no backups. " +
+                                   "This can lead to data loss and incorrect query results.");
+        }
+
+        /// <summary>
+        /// Gets the cache key to be used with GetItem and PutItem.
+        /// </summary>
+        public DbCacheKey GetCacheKey(string key, ICollection<EntitySetBase> dependentEntitySets, DbCachingMode mode)
+        {
+            if (mode == DbCachingMode.ReadWrite)
+            {
+                var versions = GetEntitySetVersions(dependentEntitySets);
+
+                return new DbCacheKey(key, dependentEntitySets, versions);
+            }
+
+            if (mode == DbCachingMode.ReadOnly)
+                return new DbCacheKey(key, null, null);
+
+            throw new ArgumentOutOfRangeException("mode");
+        }
+
+        /// <summary>
+        /// Gets the item from cache.
+        /// </summary>
+        public bool GetItem(DbCacheKey key, out object value)
+        {
+            var valueBytes = ((ICacheInternal) _cache).DoOutInOpExtension(ExtensionId, OpGetItem,
+                w => WriteKey(key, w, false), r => r.ReadObject<byte[]>());
+
+            if (valueBytes == null)
+            {
+                value = null;
+
+                return false;
+            }
+
+            using (var ms = new MemoryStream(valueBytes))
+            {
+                value = new BinaryFormatter().Deserialize(ms);
+            }
+
+            return true;
+        }
+
+        /// <summary>
+        /// Puts the item to cache.
+        /// </summary>
+        public void PutItem(DbCacheKey key, object value, TimeSpan absoluteExpiration)
+        {
+            using (var stream = new MemoryStream())
+            {
+                new BinaryFormatter().Serialize(stream, value);
+
+                var valueBytes = stream.ToArray();
+
+                var cache = GetCacheWithExpiry(absoluteExpiration);
+
+                ((ICacheInternal)cache).DoOutInOpExtension<object>(ExtensionId, OpPutItem, w =>
+                {
+                    WriteKey(key, w, true);
+
+                    w.WriteByteArray(valueBytes);
+                }, null);
+            }
+        }
+
+        /// <summary>
+        /// Invalidates the sets.
+        /// </summary>
+        public void InvalidateSets(ICollection<EntitySetBase> entitySets)
+        {
+            Debug.Assert(entitySets != null && entitySets.Count > 0);
+
+            // Increase version for each dependent entity set and run a task to clean up old entries.
+            ((ICacheInternal) _metaCache).DoOutInOpExtension<object>(ExtensionId, OpInvalidateSets, w =>
+            {
+                w.WriteString(_cache.Name);
+
+                w.WriteInt(entitySets.Count);
+
+                foreach (var set in entitySets)
+                    w.WriteString(set.Name);
+            }, null);
+        }
+
+        /// <summary>
+        /// Gets the cache with expiry policy according to provided expiration date.
+        /// </summary>
+        /// <returns>Cache with expiry policy.</returns>
+        // ReSharper disable once UnusedParameter.Local
+        private ICache<string, object> GetCacheWithExpiry(TimeSpan absoluteExpiration)
+        {
+            if (absoluteExpiration == TimeSpan.MaxValue)
+                return _cache;
+
+            // Round up to 0.1 of a second so that we share expiry caches
+            var expirySeconds = GetSeconds(absoluteExpiration);
+
+            ICache<string, object> expiryCache;
+
+            if (_expiryCaches.TryGetValue(expirySeconds, out expiryCache))
+                return expiryCache;
+
+            lock (_syncRoot)
+            {
+                if (_expiryCaches.TryGetValue(expirySeconds, out expiryCache))
+                    return expiryCache;
+
+                // Copy on write with size limit
+                _expiryCaches = _expiryCaches.Count > MaxExpiryCaches
+                    ? new Dictionary<long, ICache<string, object>>()
+                    : new Dictionary<long, ICache<string, object>>(_expiryCaches);
+
+                expiryCache =
+                    _cache.WithExpiryPolicy(GetExpiryPolicy(expirySeconds));
+
+                _expiryCaches[expirySeconds] = expiryCache;
+
+                return expiryCache;
+            }
+        }
+
+        /// <summary>
+        /// Gets the expiry policy.
+        /// </summary>
+        private static ExpiryPolicy GetExpiryPolicy(long absoluteSeconds)
+        {
+            var absolute = absoluteSeconds != long.MaxValue
+                ? TimeSpan.FromSeconds((double)absoluteSeconds / 10)
+                : (TimeSpan?) null;
+
+            return new ExpiryPolicy(absolute, null, null);
+        }
+
+        /// <summary>
+        /// Gets the seconds.
+        /// </summary>
+        private static long GetSeconds(TimeSpan ts)
+        {
+            if (ts == TimeSpan.MaxValue)
+                return long.MaxValue;
+
+            var seconds = ts.TotalSeconds;
+
+            if (seconds < 0)
+                seconds = 0;
+
+            return (long) (seconds * 10);
+        }
+
+        /// <summary>
+        /// Gets the entity set versions.
+        /// </summary>
+        private IDictionary<string, long> GetEntitySetVersions(ICollection<EntitySetBase> sets)
+        {
+            // LINQ Select allocates less that a new List<> will do.
+            var versions = _metaCache.GetAll(sets.Select(x => x.Name));
+
+            // Some versions may be missing, fill up with 0.
+            foreach (var set in sets)
+            {
+                if (!versions.ContainsKey(set.Name))
+                    versions[set.Name] = 0;
+            }
+
+            Debug.Assert(sets.Count == versions.Count);
+
+            return versions;
+        }
+
+        /// <summary>
+        /// Writes the key.
+        /// </summary>
+        private static void WriteKey(DbCacheKey key, IBinaryRawWriter writer, bool includeNames)
+        {
+            writer.WriteString(key.Key);
+
+            if (key.EntitySetVersions != null)
+            {
+                writer.WriteInt(key.EntitySetVersions.Count);
+
+                // Versions should be in the same order, so we can't iterate over the dictionary.
+                foreach (var entitySet in key.EntitySets)
+                {
+                    writer.WriteLong(key.EntitySetVersions[entitySet.Name]);
+
+                    if (includeNames)
+                        writer.WriteString(entitySet.Name);
+                }
+            }
+            else
+            {
+                writer.WriteInt(-1);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCacheKey.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCacheKey.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCacheKey.cs
new file mode 100644
index 0000000..7974ba9
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCacheKey.cs
@@ -0,0 +1,92 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.EntityFramework.Impl
+{
+    using System.Collections.Generic;
+    using System.Data.Entity.Core.Metadata.Edm;
+    using System.Diagnostics;
+
+    /// <summary>
+    /// Represents a cache key, including dependent entity sets and their versions.
+    /// </summary>
+    internal class DbCacheKey
+    {
+        /** Original string key. */
+        private readonly string _key;
+
+        /** Ordered entity sets. */
+        private readonly ICollection<EntitySetBase> _entitySets;
+
+        /** Entity set versions. */
+        private readonly IDictionary<string, long> _entitySetVersions;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="DbCacheKey"/> class.
+        /// </summary>
+        public DbCacheKey(string key, ICollection<EntitySetBase> entitySets, 
+            IDictionary<string, long> entitySetVersions)
+        {
+            Debug.Assert(key != null);
+
+            _key = key;
+            _entitySetVersions = entitySetVersions;
+            _entitySets = entitySets;
+        }
+
+        /// <summary>
+        /// Gets the key.
+        /// </summary>
+        public string Key
+        {
+            get { return _key; }
+        }
+
+        /// <summary>
+        /// Gets the entity sets.
+        /// </summary>
+        public ICollection<EntitySetBase> EntitySets
+        {
+            get { return _entitySets; }
+        }
+
+        /// <summary>
+        /// Gets the entity set versions.
+        /// </summary>
+        public IDictionary<string, long> EntitySetVersions
+        {
+            get { return _entitySetVersions; }
+        }
+
+        ///// <summary>
+        ///// Gets the versioned key.
+        ///// </summary>
+        //public void GetStringKey()
+        //{
+        //    if (_entitySetVersions == null)
+        //        return _key;
+
+        //    var sb = new StringBuilder(_key);
+
+        //    // Versions should be in the same order, so we can't iterate over the dictionary.
+        //    foreach (var entitySet in _entitySets)
+        //        sb.AppendFormat("_{0}", _entitySetVersions[entitySet.Name]);
+
+        //    return sb.ToString();
+        //}
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandDefinitionProxy.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandDefinitionProxy.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandDefinitionProxy.cs
new file mode 100644
index 0000000..7057628
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandDefinitionProxy.cs
@@ -0,0 +1,51 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.EntityFramework.Impl
+{
+    using System.Data.Common;
+    using System.Data.Entity.Core.Common;
+    using System.Diagnostics;
+
+    internal class DbCommandDefinitionProxy : DbCommandDefinition
+    {
+        /** */
+        private readonly DbCommandDefinition _definition;
+
+        /** */
+        private readonly DbCommandInfo _info;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="DbCommandDefinitionProxy"/> class.
+        /// </summary>
+        public DbCommandDefinitionProxy(DbCommandDefinition definition, DbCommandInfo info)
+        {
+            Debug.Assert(definition != null);
+
+            var proxy = definition as DbCommandDefinitionProxy;
+            _definition = proxy != null ? proxy._definition : definition;
+
+            _info = info;
+        }
+
+        /** <inheritDoc /> */
+        public override DbCommand CreateCommand()
+        {
+            return new DbCommandProxy(_definition.CreateCommand(), _info);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs
new file mode 100644
index 0000000..7f18170
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs
@@ -0,0 +1,158 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.EntityFramework.Impl
+{
+    using System.Collections.Generic;
+    using System.Data.Entity.Core.Common.CommandTrees;
+    using System.Data.Entity.Core.Metadata.Edm;
+    using System.Diagnostics;
+    using System.Linq;
+
+    /// <summary>
+    /// Command info.
+    /// </summary>
+    internal class DbCommandInfo
+    {
+        /** */
+        private readonly bool _isModification;
+
+        /** */
+        private readonly DbCache _cache;
+
+        /** */
+        private readonly EntitySetBase[] _affectedEntitySets;
+
+        /** */
+        private readonly IDbCachingPolicy _policy;
+
+        /** */
+        private readonly DbTransactionInterceptor _txHandler;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="DbCommandInfo"/> class.
+        /// </summary>
+        public DbCommandInfo(DbCommandTree tree, DbCache cache, IDbCachingPolicy policy, DbTransactionInterceptor txHandler)
+        {
+            Debug.Assert(tree != null);
+            Debug.Assert(cache != null);
+            Debug.Assert(txHandler != null);
+
+            var qryTree = tree as DbQueryCommandTree;
+
+            if (qryTree != null)
+            {
+                _isModification = false;
+
+                _affectedEntitySets = GetAffectedEntitySets(qryTree.Query);
+            }
+            else
+            {
+                _isModification = true;
+
+                var modify = tree as DbModificationCommandTree;
+
+                if (modify != null)
+                    _affectedEntitySets = GetAffectedEntitySets(modify.Target.Expression);
+                else
+                    // Functions (stored procedures) are not supported.
+                    Debug.Assert(tree is DbFunctionCommandTree);
+            }
+
+            _cache = cache;
+            _policy = policy;
+            _txHandler = txHandler;
+        }
+
+        /// <summary>
+        /// Gets a value indicating whether this command is a query and does not modify data.
+        /// </summary>
+        public bool IsModification
+        {
+            get { return _isModification; }
+        }
+
+        /// <summary>
+        /// Gets or sets the cache.
+        /// </summary>
+        public DbCache Cache
+        {
+            get { return _cache; }
+        }
+
+        /// <summary>
+        /// Gets the affected entity sets.
+        /// </summary>
+        public ICollection<EntitySetBase> AffectedEntitySets
+        {
+            get { return _affectedEntitySets; }
+        }
+
+        /// <summary>
+        /// Gets the policy.
+        /// </summary>
+        public IDbCachingPolicy Policy
+        {
+            get { return _policy; }
+        }
+
+        /// <summary>
+        /// Gets the tx handler.
+        /// </summary>
+        public DbTransactionInterceptor TxHandler
+        {
+            get { return _txHandler; }
+        }
+
+        /// <summary>
+        /// Gets the affected entity sets.
+        /// </summary>
+        private static EntitySetBase[] GetAffectedEntitySets(DbExpression expression)
+        {
+            var visitor = new ScanExpressionVisitor();
+
+            expression.Accept(visitor);
+
+            return visitor.EntitySets.ToArray();
+        }
+
+        /// <summary>
+        /// Visits Scan expressions and collects entity set names.
+        /// </summary>
+        private class ScanExpressionVisitor : BasicCommandTreeVisitor
+        {
+            /** */
+            private readonly List<EntitySetBase> _entitySets = new List<EntitySetBase>();
+
+            /// <summary>
+            /// Gets the entity sets.
+            /// </summary>
+            public IEnumerable<EntitySetBase> EntitySets
+            {
+                get { return _entitySets; }
+            }
+
+            /** <inheritdoc /> */
+            public override void Visit(DbScanExpression expression)
+            {
+                _entitySets.Add(expression.Target);
+
+                base.Visit(expression);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandProxy.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandProxy.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandProxy.cs
new file mode 100644
index 0000000..e3353d5
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandProxy.cs
@@ -0,0 +1,263 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.EntityFramework.Impl
+{
+    using System;
+    using System.Data;
+    using System.Data.Common;
+    using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Text;
+
+    /// <summary>
+    /// Command proxy.
+    /// </summary>
+    internal class DbCommandProxy : DbCommand
+    {
+        /** */
+        private readonly DbCommand _command;
+
+        /** */
+        private readonly DbCommandInfo _commandInfo;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="DbCommandProxy"/> class.
+        /// </summary>
+        public DbCommandProxy(DbCommand command, DbCommandInfo info)
+        {
+            Debug.Assert(command != null);
+            Debug.Assert(info != null);
+
+            _command = command;
+            _commandInfo = info;
+        }
+
+        /// <summary>
+        /// Gets the inner command.
+        /// </summary>
+        [ExcludeFromCodeCoverage]
+        public DbCommand InnerCommand
+        {
+            get { return _command; }
+        }
+
+        /// <summary>
+        /// Gets the command information.
+        /// </summary>
+        [ExcludeFromCodeCoverage]
+        public DbCommandInfo CommandInfo
+        {
+            get { return _commandInfo; }
+        }
+
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        public override void Prepare()
+        {
+            _command.Prepare();
+        }
+
+        /** <inheritDoc /> */
+        public override string CommandText
+        {
+            get { return _command.CommandText; }
+            set { _command.CommandText = value; }
+        }
+
+        /** <inheritDoc /> */
+        public override int CommandTimeout
+        {
+            get { return _command.CommandTimeout; }
+            set { _command.CommandTimeout = value; }
+        }
+
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        public override CommandType CommandType
+        {
+            get { return _command.CommandType; }
+            set { _command.CommandType = value; }
+        }
+
+        /** <inheritDoc /> */
+        public override UpdateRowSource UpdatedRowSource
+        {
+            get { return _command.UpdatedRowSource; }
+            set { _command.UpdatedRowSource = value; }
+        }
+
+        /** <inheritDoc /> */
+        protected override DbConnection DbConnection
+        {
+            get { return _command.Connection; }
+            set { _command.Connection = value; }
+        }
+
+        /** <inheritDoc /> */
+        protected override DbParameterCollection DbParameterCollection
+        {
+            get { return _command.Parameters; }
+        }
+
+        /** <inheritDoc /> */
+        protected override DbTransaction DbTransaction
+        {
+            get { return _command.Transaction; }
+            set { _command.Transaction = value; }
+        }
+
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        public override bool DesignTimeVisible
+        {
+            get { return _command.DesignTimeVisible; }
+            set { _command.DesignTimeVisible = value; }
+        }
+
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        public override void Cancel()
+        {
+            _command.Cancel();
+        }
+
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        protected override DbParameter CreateDbParameter()
+        {
+            return _command.CreateParameter();
+        }
+
+        /** <inheritDoc /> */
+        protected override DbDataReader ExecuteDbDataReader(CommandBehavior behavior)
+        {
+            if (_commandInfo.IsModification)
+            {
+                // Execute reader, then invalidate cached data.
+                var dbReader = _command.ExecuteReader(behavior);
+
+                InvalidateCache();
+
+                return dbReader;
+            }
+
+            if (Transaction != null)
+            {
+                return _command.ExecuteReader(behavior);
+            }
+
+            var queryInfo = GetQueryInfo();
+            var strategy = _commandInfo.Policy.GetCachingMode(queryInfo);
+            var cacheKey = _commandInfo.Cache.GetCacheKey(GetKey(), _commandInfo.AffectedEntitySets, strategy);
+
+            object cachedRes;
+            if (_commandInfo.Cache.GetItem(cacheKey, out cachedRes))
+                return ((DataReaderResult) cachedRes).CreateReader();
+
+            var reader = _command.ExecuteReader(behavior);
+
+            if (reader.RecordsAffected > 0)
+                return reader;  // Queries that modify anything are never cached.
+
+            // Check if cacheable.
+            if (!_commandInfo.Policy.CanBeCached(queryInfo))
+                return reader;
+
+            // Read into memory.
+            var res = new DataReaderResult(reader);
+
+            // Check if specific row count is cacheable.
+            if (!_commandInfo.Policy.CanBeCached(queryInfo, res.RowCount))
+                return res.CreateReader();
+
+            PutResultToCache(cacheKey, res, queryInfo);
+
+            return res.CreateReader();
+        }
+
+        /// <summary>
+        /// Invalidates the cache.
+        /// </summary>
+        private void InvalidateCache()
+        {
+            _commandInfo.TxHandler.InvalidateCache(_commandInfo.AffectedEntitySets, Transaction);
+        }
+
+        /** <inheritDoc /> */
+        public override int ExecuteNonQuery()
+        {
+            var res = _command.ExecuteNonQuery();
+
+            // Invalidate AFTER updating the data.
+            if (_commandInfo.IsModification)
+            {
+                InvalidateCache();
+            }
+
+            return res;
+        }
+
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        public override object ExecuteScalar()
+        {
+            // This method is never used by EntityFramework.
+            // Even EntityCommand.ExecuteScalar goes to ExecuteDbDataReader.
+            return _command.ExecuteScalar();
+        }
+
+        /// <summary>
+        /// Puts the result to cache.
+        /// </summary>
+        private void PutResultToCache(DbCacheKey key, object result, DbQueryInfo queryInfo)
+        {
+            var expiration = _commandInfo.Policy != null
+                ? _commandInfo.Policy.GetExpirationTimeout(queryInfo)
+                : TimeSpan.MaxValue;
+
+            _commandInfo.Cache.PutItem(key, result, expiration);
+        }
+
+        /// <summary>
+        /// Gets the cache key.
+        /// </summary>
+        private string GetKey()
+        {
+            if (string.IsNullOrEmpty(CommandText))
+                throw new NotSupportedException("Ignite Entity Framework Caching " +
+                                                "requires non-empty DbCommand.CommandText.");
+
+            var sb = new StringBuilder();
+
+            sb.AppendFormat("{0}:{1}|", Connection.Database, CommandText);
+
+            foreach (DbParameter param in Parameters)
+                sb.AppendFormat("{0}={1},", param.ParameterName, param.Value);
+
+            return sb.ToString();
+        }
+
+        /// <summary>
+        /// Gets the query information.
+        /// </summary>
+        private DbQueryInfo GetQueryInfo()
+        {
+            return new DbQueryInfo(_commandInfo.AffectedEntitySets, CommandText, DbParameterCollection);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbProviderServicesProxy.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbProviderServicesProxy.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbProviderServicesProxy.cs
new file mode 100644
index 0000000..8e01295
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbProviderServicesProxy.cs
@@ -0,0 +1,169 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma warning disable 618, 672
+namespace Apache.Ignite.EntityFramework.Impl
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Data.Common;
+    using System.Data.Entity.Core.Common;
+    using System.Data.Entity.Core.Common.CommandTrees;
+    using System.Data.Entity.Core.Metadata.Edm;
+    using System.Data.Entity.Spatial;
+    using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
+
+    /// <summary>
+    /// DbProviderServices proxy which substitutes custom commands.
+    /// </summary>
+    internal class DbProviderServicesProxy : DbProviderServices
+    {
+        /** */
+        private static readonly DbCachingPolicy DefaultPolicy = new DbCachingPolicy();
+
+        /** */
+        private readonly IDbCachingPolicy _policy;
+        
+        /** */
+        private readonly DbProviderServices _services;
+        
+        /** */
+        private readonly DbCache _cache;
+
+        /** */
+        private readonly DbTransactionInterceptor _txHandler;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="DbProviderServicesProxy"/> class.
+        /// </summary>
+        /// <param name="services">The services.</param>
+        /// <param name="policy">The policy.</param>
+        /// <param name="cache">The cache.</param>
+        /// <param name="txHandler">Transaction handler.</param>
+        public DbProviderServicesProxy(DbProviderServices services, IDbCachingPolicy policy, DbCache cache, 
+            DbTransactionInterceptor txHandler)
+        {
+            Debug.Assert(services != null);
+            Debug.Assert(cache != null);
+            Debug.Assert(txHandler != null);
+
+            var proxy = services as DbProviderServicesProxy;
+            _services = proxy != null ? proxy._services : services;
+
+            _policy = policy ?? DefaultPolicy;
+            _cache = cache;
+            _txHandler = txHandler;
+        }
+
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        public override DbCommandDefinition CreateCommandDefinition(DbCommand prototype)
+        {
+            var proxy = prototype as DbCommandProxy;
+
+            if (proxy == null)
+                return _services.CreateCommandDefinition(prototype);
+
+            return new DbCommandDefinitionProxy(_services.CreateCommandDefinition(proxy.InnerCommand), 
+                proxy.CommandInfo);
+        }
+
+        /** <inheritDoc /> */
+        protected override DbCommandDefinition CreateDbCommandDefinition(DbProviderManifest providerManifest, 
+            DbCommandTree commandTree)
+        {
+            return new DbCommandDefinitionProxy(_services.CreateCommandDefinition(providerManifest, commandTree), 
+                new DbCommandInfo(commandTree, _cache, _policy, _txHandler));
+        }
+
+        /** <inheritDoc /> */
+        protected override string GetDbProviderManifestToken(DbConnection connection)
+        {
+            return _services.GetProviderManifestToken(connection);
+        }
+
+        /** <inheritDoc /> */
+        protected override DbProviderManifest GetDbProviderManifest(string manifestToken)
+        {
+            return _services.GetProviderManifest(manifestToken);
+        }
+
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        public override void RegisterInfoMessageHandler(DbConnection connection, Action<string> handler)
+        {
+            _services.RegisterInfoMessageHandler(connection, handler);
+        }
+
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        protected override DbSpatialDataReader GetDbSpatialDataReader(DbDataReader fromReader, string manifestToken)
+        {
+            return _services.GetSpatialDataReader(fromReader, manifestToken);
+        }
+
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        protected override DbSpatialServices DbGetSpatialServices(string manifestToken)
+        {
+            return _services.GetSpatialServices(manifestToken);
+        }
+        protected override void SetDbParameterValue(DbParameter parameter, TypeUsage parameterType, object value)
+        {
+            _services.SetParameterValue(parameter, parameterType, value);
+        }
+
+        /** <inheritDoc /> */
+        protected override string DbCreateDatabaseScript(string providerManifestToken, StoreItemCollection storeItemCollection)
+        {
+            return _services.CreateDatabaseScript(providerManifestToken, storeItemCollection);
+        }
+
+        /** <inheritDoc /> */
+        protected override void DbCreateDatabase(DbConnection connection, int? commandTimeout, StoreItemCollection storeItemCollection)
+        {
+            _services.CreateDatabase(connection, commandTimeout, storeItemCollection);
+        }
+
+        /** <inheritDoc /> */
+        protected override bool DbDatabaseExists(DbConnection connection, int? commandTimeout, StoreItemCollection storeItemCollection)
+        {
+            return _services.DatabaseExists(connection, commandTimeout, storeItemCollection);
+        }
+
+        /** <inheritDoc /> */
+        protected override void DbDeleteDatabase(DbConnection connection, int? commandTimeout, StoreItemCollection storeItemCollection)
+        {
+            _services.DeleteDatabase(connection, commandTimeout, storeItemCollection);
+        }
+
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        public override object GetService(Type type, object key)
+        {
+            return _services.GetService(type, key);
+        }
+
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        public override IEnumerable<object> GetServices(Type type, object key)
+        {
+            return _services.GetServices(type, key);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbTransactionInterceptor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbTransactionInterceptor.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbTransactionInterceptor.cs
new file mode 100644
index 0000000..601868e
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbTransactionInterceptor.cs
@@ -0,0 +1,134 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.EntityFramework.Impl
+{
+    using System.Collections.Concurrent;
+    using System.Collections.Generic;
+    using System.Data;
+    using System.Data.Common;
+    using System.Data.Entity.Core.Metadata.Edm;
+    using System.Data.Entity.Infrastructure.Interception;
+    using System.Diagnostics.CodeAnalysis;
+
+    /// <summary>
+    /// Intercepts transaction events.
+    /// </summary>
+    internal class DbTransactionInterceptor : IDbTransactionInterceptor
+    {
+        /** Cache. */
+        private readonly DbCache _cache;
+
+        /** Map from tx to dependent sets. HashSet because same sets can be affected multiple times within a tx. */
+        private readonly ConcurrentDictionary<DbTransaction, HashSet<EntitySetBase>> _entitySets 
+            = new ConcurrentDictionary<DbTransaction, HashSet<EntitySetBase>>();
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="DbTransactionInterceptor"/> class.
+        /// </summary>
+        /// <param name="cache">The cache.</param>
+        public DbTransactionInterceptor(DbCache cache)
+        {
+            _cache = cache;
+        }
+
+        /** <inheritDoc /> */
+        public void InvalidateCache(ICollection<EntitySetBase> entitySets, DbTransaction transaction)
+        {
+            if (transaction == null)
+            {
+                // Invalidate immediately.
+                _cache.InvalidateSets(entitySets);
+            }
+            else
+            {
+                // Postpone until commit.
+                var sets = _entitySets.GetOrAdd(transaction, _ => new HashSet<EntitySetBase>());
+
+                foreach (var set in entitySets)
+                    sets.Add(set);
+            }
+        }
+
+        /** <inheritDoc /> */
+        public void ConnectionGetting(DbTransaction transaction, DbTransactionInterceptionContext<DbConnection> interceptionContext)
+        {
+            // No-op
+        }
+
+        /** <inheritDoc /> */
+        public void ConnectionGot(DbTransaction transaction, DbTransactionInterceptionContext<DbConnection> interceptionContext)
+        {
+            // No-op
+        }
+
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        public void IsolationLevelGetting(DbTransaction transaction, DbTransactionInterceptionContext<IsolationLevel> interceptionContext)
+        {
+            // No-op
+        }
+
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        public void IsolationLevelGot(DbTransaction transaction, DbTransactionInterceptionContext<IsolationLevel> interceptionContext)
+        {
+            // No-op
+        }
+
+        /** <inheritDoc /> */
+        public void Committing(DbTransaction transaction, DbTransactionInterceptionContext interceptionContext)
+        {
+            // No-op
+        }
+
+        /** <inheritDoc /> */
+        public void Committed(DbTransaction transaction, DbTransactionInterceptionContext interceptionContext)
+        {
+            HashSet<EntitySetBase> entitySets;
+            if (_entitySets.TryGetValue(transaction, out entitySets))
+                _cache.InvalidateSets(entitySets);
+        }
+
+        /** <inheritDoc /> */
+        public void Disposing(DbTransaction transaction, DbTransactionInterceptionContext interceptionContext)
+        {
+            // No-op
+        }
+
+        /** <inheritDoc /> */
+        public void Disposed(DbTransaction transaction, DbTransactionInterceptionContext interceptionContext)
+        {
+            HashSet<EntitySetBase> val;
+            _entitySets.TryRemove(transaction, out val);
+        }
+
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        public void RollingBack(DbTransaction transaction, DbTransactionInterceptionContext interceptionContext)
+        {
+            // No-op
+        }
+
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        public void RolledBack(DbTransaction transaction, DbTransactionInterceptionContext interceptionContext)
+        {
+            // No-op
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Properties/AssemblyInfo.cs
new file mode 100644
index 0000000..7ce4c5f
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Properties/AssemblyInfo.cs
@@ -0,0 +1,41 @@
+\ufeff/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*      http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+using System;
+using System.Reflection;
+using System.Runtime.CompilerServices;
+using System.Runtime.InteropServices;
+
+[assembly: AssemblyTitle("Apache.Ignite.EntityFramework")]
+[assembly: AssemblyDescription("Apache Ignite.NET EntityFramework integration")]
+[assembly: AssemblyConfiguration("")]
+[assembly: AssemblyCompany("Apache Software Foundation")]
+[assembly: AssemblyProduct("Apache Ignite.NET")]
+[assembly: AssemblyCopyright("Copyright �  2015")]
+[assembly: AssemblyTrademark("")]
+[assembly: AssemblyCulture("")]
+
+[assembly: ComVisible(false)]
+[assembly: Guid("c558518a-c1a0-4224-aaa9-a8688474b4dc")]
+
+[assembly: AssemblyVersion("1.8.0.14218")]
+[assembly: AssemblyFileVersion("1.8.0.14218")]
+[assembly: AssemblyInformationalVersion("1.8.0")]
+
+[assembly: CLSCompliant(true)]
+
+[assembly: InternalsVisibleTo("Apache.Ignite.EntityFramework.Tests, PublicKey=00240000048000009400000006020000002400005253413100040000010001005f45ca91396d3bb682c38d96bdc6e9ac5855a2b8f7dd7434493c278ceb75cae29d452714a376221e5bfc26dfc7dadcdbe9d0a8bb04b1945f6c326089481fc65da5fa8fc728fa9dde5fa2e1599f89678c6b1b38c59d5deef7d012eced64941d5d065aff987ec0196f5b352213d5c04b982647d7fb3bfb2496b890afc5ef1391b0")]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/packages.config
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/packages.config b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/packages.config
new file mode 100644
index 0000000..c623cae
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/packages.config
@@ -0,0 +1,20 @@
+\ufeff<?xml version="1.0" encoding="utf-8"?>
+<!--
+  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.
+-->
+<packages>
+  <package id="EntityFramework" version="6.1.3" targetFramework="net40" />
+</packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.sln
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.sln b/modules/platforms/dotnet/Apache.Ignite.sln
index de7cf19..fed0821 100644
--- a/modules/platforms/dotnet/Apache.Ignite.sln
+++ b/modules/platforms/dotnet/Apache.Ignite.sln
@@ -42,6 +42,10 @@ Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.AspNet.Tests"
 EndProject
 Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.Log4Net", "Apache.Ignite.log4net\Apache.Ignite.Log4Net.csproj", "{6F82D669-382E-4435-8092-68C4440146D8}"
 EndProject
+Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.EntityFramework", "Apache.Ignite.EntityFramework\Apache.Ignite.EntityFramework.csproj", "{C558518A-C1A0-4224-AAA9-A8688474B4DC}"
+EndProject
+Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.EntityFramework.Tests", "Apache.Ignite.EntityFramework.Tests\Apache.Ignite.EntityFramework.Tests.csproj", "{CDA5700E-78F3-4A9E-A9B0-704CBE94651C}"
+EndProject
 Global
 	GlobalSection(SolutionConfigurationPlatforms) = preSolution
 		Debug|Any CPU = Debug|Any CPU
@@ -216,6 +220,30 @@ Global
 		{6F82D669-382E-4435-8092-68C4440146D8}.Release|x64.Build.0 = Release|Any CPU
 		{6F82D669-382E-4435-8092-68C4440146D8}.Release|x86.ActiveCfg = Release|Any CPU
 		{6F82D669-382E-4435-8092-68C4440146D8}.Release|x86.Build.0 = Release|Any CPU
+		{C558518A-C1A0-4224-AAA9-A8688474B4DC}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
+		{C558518A-C1A0-4224-AAA9-A8688474B4DC}.Debug|Any CPU.Build.0 = Debug|Any CPU
+		{C558518A-C1A0-4224-AAA9-A8688474B4DC}.Debug|x64.ActiveCfg = Debug|Any CPU
+		{C558518A-C1A0-4224-AAA9-A8688474B4DC}.Debug|x64.Build.0 = Debug|Any CPU
+		{C558518A-C1A0-4224-AAA9-A8688474B4DC}.Debug|x86.ActiveCfg = Debug|Any CPU
+		{C558518A-C1A0-4224-AAA9-A8688474B4DC}.Debug|x86.Build.0 = Debug|Any CPU
+		{C558518A-C1A0-4224-AAA9-A8688474B4DC}.Release|Any CPU.ActiveCfg = Release|Any CPU
+		{C558518A-C1A0-4224-AAA9-A8688474B4DC}.Release|Any CPU.Build.0 = Release|Any CPU
+		{C558518A-C1A0-4224-AAA9-A8688474B4DC}.Release|x64.ActiveCfg = Release|Any CPU
+		{C558518A-C1A0-4224-AAA9-A8688474B4DC}.Release|x64.Build.0 = Release|Any CPU
+		{C558518A-C1A0-4224-AAA9-A8688474B4DC}.Release|x86.ActiveCfg = Release|Any CPU
+		{C558518A-C1A0-4224-AAA9-A8688474B4DC}.Release|x86.Build.0 = Release|Any CPU
+		{CDA5700E-78F3-4A9E-A9B0-704CBE94651C}.Debug|Any CPU.ActiveCfg = Debug|Any CPU
+		{CDA5700E-78F3-4A9E-A9B0-704CBE94651C}.Debug|Any CPU.Build.0 = Debug|Any CPU
+		{CDA5700E-78F3-4A9E-A9B0-704CBE94651C}.Debug|x64.ActiveCfg = Debug|Any CPU
+		{CDA5700E-78F3-4A9E-A9B0-704CBE94651C}.Debug|x64.Build.0 = Debug|Any CPU
+		{CDA5700E-78F3-4A9E-A9B0-704CBE94651C}.Debug|x86.ActiveCfg = Debug|Any CPU
+		{CDA5700E-78F3-4A9E-A9B0-704CBE94651C}.Debug|x86.Build.0 = Debug|Any CPU
+		{CDA5700E-78F3-4A9E-A9B0-704CBE94651C}.Release|Any CPU.ActiveCfg = Release|Any CPU
+		{CDA5700E-78F3-4A9E-A9B0-704CBE94651C}.Release|Any CPU.Build.0 = Release|Any CPU
+		{CDA5700E-78F3-4A9E-A9B0-704CBE94651C}.Release|x64.ActiveCfg = Release|Any CPU
+		{CDA5700E-78F3-4A9E-A9B0-704CBE94651C}.Release|x64.Build.0 = Release|Any CPU
+		{CDA5700E-78F3-4A9E-A9B0-704CBE94651C}.Release|x86.ActiveCfg = Release|Any CPU
+		{CDA5700E-78F3-4A9E-A9B0-704CBE94651C}.Release|x86.Build.0 = Release|Any CPU
 	EndGlobalSection
 	GlobalSection(SolutionProperties) = preSolution
 		HideSolutionNode = FALSE


[46/50] [abbrv] ignite git commit: Merge branch 'ignite-1.7.4' into master

Posted by vo...@apache.org.
Merge branch 'ignite-1.7.4' into master


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

Branch: refs/heads/ignite-2693
Commit: ad2831e784cb15d1a106cbded794d2e075389b93
Parents: e007f6e d69e26d
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Nov 14 12:14:19 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Nov 14 12:14:19 2016 +0300

----------------------------------------------------------------------
 .../platform/PlatformAbstractTarget.java        |  65 +---
 .../platform/PlatformNoopProcessor.java         |   5 +
 .../processors/platform/PlatformProcessor.java  |   7 +
 .../platform/PlatformProcessorImpl.java         |  21 +-
 .../processors/platform/PlatformTarget.java     |  20 --
 .../binary/PlatformBinaryProcessor.java         |  96 ++++++
 .../platform/cache/PlatformCache.java           |  48 ++-
 .../cache/affinity/PlatformAffinity.java        |   4 +-
 .../query/PlatformAbstractQueryCursor.java      |   4 +-
 .../query/PlatformContinuousQueryProxy.java     |   3 +-
 .../platform/cluster/PlatformClusterGroup.java  |  35 +-
 .../platform/compute/PlatformCompute.java       |   9 +-
 .../datastreamer/PlatformDataStreamer.java      |   9 +-
 .../datastructures/PlatformAtomicLong.java      |  27 +-
 .../datastructures/PlatformAtomicReference.java |   8 +-
 .../datastructures/PlatformAtomicSequence.java  |  31 +-
 .../platform/services/PlatformServices.java     |  13 +-
 .../transactions/PlatformTransactions.java      |  17 +-
 .../utils/PlatformConfigurationUtils.java       |  63 +++-
 modules/hadoop/pom.xml                          |  49 +++
 .../query/h2/sql/GridSqlQuerySplitter.java      | 143 +++++---
 .../query/IgniteSqlSplitterSelfTest.java        | 326 +++++++++++++++++++
 .../ignite/impl/binary/binary_type_manager.h    |  14 +-
 .../impl/binary/binary_type_updater_impl.h      |  12 +-
 .../include/ignite/impl/ignite_environment.h    |  49 ++-
 modules/platforms/cpp/core/src/ignition.cpp     |   8 +-
 .../impl/binary/binary_type_updater_impl.cpp    |  13 +-
 .../core/src/impl/cache/query/query_impl.cpp    |   6 +-
 .../cpp/core/src/impl/ignite_environment.cpp    |  45 ++-
 .../core/src/impl/interop/interop_target.cpp    |   6 +-
 .../cpp/jni/include/ignite/jni/exports.h        |   3 +-
 .../platforms/cpp/jni/include/ignite/jni/java.h |   6 +-
 .../cpp/jni/include/ignite/jni/utils.h          |  91 +++++-
 modules/platforms/cpp/jni/project/vs/module.def |   6 +-
 modules/platforms/cpp/jni/src/exports.cpp       |  12 +-
 modules/platforms/cpp/jni/src/java.cpp          |  36 +-
 .../Apache.Ignite.AspNet.csproj                 |   3 +-
 .../Apache.Ignite.Core.Tests.csproj             |   1 +
 .../Cache/CacheAbstractTest.cs                  |  46 ++-
 .../Cache/CacheSwapSpaceTest.cs                 | 120 +++++++
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |   3 +
 .../IgniteConfigurationSerializerTest.cs        |  21 +-
 .../IgniteConfigurationTest.cs                  |  24 +-
 .../ProjectFilesTest.cs                         |  62 ++++
 .../Apache.Ignite.Core.Tests/ReconnectTest.cs   |  81 ++++-
 .../Apache.Ignite.Core.csproj                   |   7 +-
 .../Binary/BinaryConfiguration.cs               |   5 +-
 .../Cache/Configuration/CacheConfiguration.cs   |   2 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  36 +-
 .../IgniteConfigurationSection.xsd              |  46 ++-
 .../Apache.Ignite.Core/Impl/Binary/Binary.cs    |   2 +-
 .../Impl/Binary/BinaryObjectSchemaSerializer.cs |   2 +-
 .../Impl/Binary/BinaryProcessor.cs              | 156 +++++++++
 .../Impl/Binary/BinaryReader.cs                 |   2 +-
 .../Impl/Binary/BinaryUtils.cs                  |  12 +
 .../Impl/Binary/BinaryWriter.cs                 |   6 +-
 .../Impl/Binary/Marshaller.cs                   |  11 +-
 .../Impl/Cache/CacheAffinityImpl.cs             |   2 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |  12 +-
 .../Impl/Cache/Query/AbstractQueryCursor.cs     |   4 +-
 .../Continuous/ContinuousQueryHandleImpl.cs     |   2 +-
 .../Impl/Cluster/ClusterGroupImpl.cs            |  72 +---
 .../Impl/Cluster/IClusterGroupEx.cs             |  35 --
 .../Impl/Compute/ComputeImpl.cs                 |   4 +-
 .../Impl/DataStructures/AtomicLong.cs           |  14 +-
 .../Impl/DataStructures/AtomicReference.cs      |   4 +-
 .../Impl/DataStructures/AtomicSequence.cs       |  14 +-
 .../Impl/Datastream/DataStreamerImpl.cs         |  20 +-
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |   4 +-
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |   1 +
 .../Impl/Handle/HandleRegistry.cs               |   8 +-
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |  29 +-
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |   9 +-
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   | 105 +-----
 .../Impl/Services/Services.cs                   |   4 +-
 .../Impl/SwapSpace/SwapSpaceSerializer.cs       |  99 ++++++
 .../Impl/Transactions/TransactionsImpl.cs       |  12 +-
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |  12 +-
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |  10 -
 .../Impl/Unmanaged/UnmanagedUtils.cs            |  19 +-
 .../SwapSpace/File/FileSwapSpaceSpi.cs          | 105 ++++++
 .../SwapSpace/ISwapSpaceSpi.cs                  |  35 ++
 .../TransactionDeadlockException.cs             |  71 ++++
 .../Apache.Ignite.Linq.csproj                   |   2 +-
 .../Apache.Ignite.Log4Net.csproj                |   4 +-
 .../Apache.Ignite.NLog.csproj                   |   4 +-
 modules/platforms/dotnet/README.txt             |  13 +-
 .../Apache.Ignite.Examples.csproj               |   1 +
 .../examples/Apache.Ignite.Examples/App.config  |   1 +
 .../Datagrid/OptimisticTransactionExample.cs    | 110 +++++++
 .../Datagrid/QueryExample.cs                    |  24 +-
 .../Apache.Ignite.ExamplesDll.csproj            |   1 +
 .../Binary/Employee.cs                          |   5 +-
 .../Compute/AverageSalaryJob.cs                 |   5 +-
 .../Datagrid/ScanQueryFilter.cs                 |  50 +++
 95 files changed, 2173 insertions(+), 711 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ad2831e7/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad2831e7/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad2831e7/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
----------------------------------------------------------------------
diff --cc modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
index 1d2e001,ac3e553..898f12a
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@@ -114,9 -114,11 +115,10 @@@ namespace Apache.Ignite.Core.Test
                              <atomicConfiguration backups='2' cacheMode='Local' atomicSequenceReserveSize='250' />
                              <transactionConfiguration defaultTransactionConcurrency='Optimistic' defaultTransactionIsolation='RepeatableRead' defaultTimeout='0:1:2' pessimisticTransactionLogSize='15' pessimisticTransactionLogLinger='0:0:33' />
                              <logger type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+TestLogger, Apache.Ignite.Core.Tests' />
+                             <swapSpaceSpi type='FileSwapSpaceSpi' baseDirectory='abcd' maximumSparsity='0.7' maximumWriteQueueSize='25' readStripesNumber='36' writeBufferSize='47' />
                          </igniteConfig>";
 -            var reader = XmlReader.Create(new StringReader(xml));
  
 -            var cfg = IgniteConfigurationXmlSerializer.Deserialize(reader);
 +            var cfg = IgniteConfiguration.FromXml(xml);
  
              Assert.AreEqual("c:", cfg.WorkDirectory);
              Assert.AreEqual("127.1.1.1", cfg.Localhost);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad2831e7/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad2831e7/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad2831e7/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad2831e7/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/HandleRegistry.cs
----------------------------------------------------------------------
diff --cc modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/HandleRegistry.cs
index 36b5072,4e1135a..588d608
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/HandleRegistry.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Handle/HandleRegistry.cs
@@@ -102,17 -102,6 +102,17 @@@ namespace Apache.Ignite.Core.Impl.Handl
          }
  
          /// <summary>
-         /// Allocate a handle for critical resource in safe mode.		
-         /// </summary>		
-         /// <param name="target">Target.</param>		
-         /// <returns>Pointer.</returns>		
++        /// Allocate a handle for critical resource in safe mode.
++        /// </summary>
++        /// <param name="target">Target.</param>
++        /// <returns>Pointer.</returns>
 +        [ExcludeFromCodeCoverage]
 +        public long AllocateCriticalSafe(object target)
 +        {
 +            return Allocate0(target, true, true);
 +        }
 +
 +        /// <summary>
          /// Internal allocation routine.
          /// </summary>
          /// <param name="target">Target.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad2831e7/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad2831e7/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad2831e7/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.csproj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad2831e7/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs
----------------------------------------------------------------------
diff --cc modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs
index 4cff2a8,b746bdf..d4637e3
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Binary/Employee.cs
@@@ -35,7 -33,8 +33,8 @@@ namespace Apache.Ignite.ExamplesDll.Bin
          /// <param name="salary">Salary.</param>
          /// <param name="address">Address.</param>
          /// <param name="departments">Departments.</param>
-         public Employee(string name, long salary, Address address, ICollection<string> departments, 
+         /// <param name="organizationId">The organization identifier.</param>
 -        public Employee(string name, long salary, Address address, ICollection<string> departments, 
++        public Employee(string name, long salary, Address address, ICollection<string> departments,
              int organizationId = 0)
          {
              Name = name;


[34/50] [abbrv] ignite git commit: IGNITE-1915 .NET: Ignite as Entity Framework Second-Level Cache

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheTest.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheTest.cs
new file mode 100644
index 0000000..cfc9f66
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheTest.cs
@@ -0,0 +1,942 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// ReSharper disable UnusedMember.Local
+// ReSharper disable UnusedAutoPropertyAccessor.Local
+// ReSharper disable ClassWithVirtualMembersNeverInherited.Local
+// ReSharper disable UnusedAutoPropertyAccessor.Global
+// ReSharper disable VirtualMemberNeverOverridden.Global
+
+namespace Apache.Ignite.EntityFramework.Tests
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Data;
+    using System.Data.Entity;
+    using System.Data.Entity.Core.EntityClient;
+    using System.Data.Entity.Infrastructure;
+    using System.IO;
+    using System.Linq;
+    using System.Threading;
+    using System.Transactions;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Tests;
+    using Apache.Ignite.EntityFramework;
+    using Apache.Ignite.EntityFramework.Impl;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Integration test with temporary SQL CE database.
+    /// </summary>
+    public class EntityFrameworkCacheTest
+    {
+        /** */
+        private static readonly string TempFile = Path.GetTempFileName();
+
+        /** */
+        private static readonly string ConnectionString = "Datasource = " + TempFile;
+
+        /** */
+        private static readonly DelegateCachingPolicy Policy = new DelegateCachingPolicy();
+
+        /** */
+        private ICache<object, object> _cache;
+
+        /** */
+        private ICache<object, object> _metaCache;
+
+        /// <summary>
+        /// Fixture set up.
+        /// </summary>
+        [TestFixtureSetUp]
+        public void FixtureSetUp()
+        {
+            // Start 2 nodes.
+            var cfg = TestUtils.GetTestConfiguration();
+            var ignite = Ignition.Start(cfg);
+
+            Ignition.Start(new IgniteConfiguration(cfg) {GridName = "grid2"});
+
+            // Create SQL CE database in a temp file.
+            using (var ctx = GetDbContext())
+            {
+                File.Delete(TempFile);
+                ctx.Database.Create();
+            }
+
+            // Get the caches.
+            _cache = ignite.GetCache<object, object>("entityFrameworkQueryCache_data")
+                .WithKeepBinary<object, object>();
+
+            _metaCache = ignite.GetCache<object, object>("entityFrameworkQueryCache_metadata")
+                .WithKeepBinary<object, object>();
+        }
+
+        /// <summary>
+        /// Fixture tear down.
+        /// </summary>
+        [TestFixtureTearDown]
+        public void FixtureTearDown()
+        {
+            using (var ctx = GetDbContext())
+            {
+                ctx.Database.Delete();
+            }
+
+            Ignition.StopAll(true);
+            File.Delete(TempFile);
+        }
+
+        /// <summary>
+        /// Sets up the test.
+        /// </summary>
+        [SetUp]
+        public void TestSetUp()
+        {
+            // Reset the policy.
+            Policy.CanBeCachedFunc = null;
+            Policy.CanBeCachedRowsFunc = null;
+            Policy.GetExpirationTimeoutFunc = null;
+            Policy.GetCachingStrategyFunc = null;
+
+            // Clean up the db.
+            using (var ctx = GetDbContext())
+            {
+                ctx.Blogs.RemoveRange(ctx.Blogs);
+                ctx.Posts.RemoveRange(ctx.Posts);
+                ctx.Tests.RemoveRange(ctx.Tests);
+
+                ctx.SaveChanges();
+            }
+
+            using (var ctx = GetDbContext())
+            {
+                Assert.IsEmpty(ctx.Blogs);
+                Assert.IsEmpty(ctx.Posts);
+            }
+
+            // Clear the caches.
+            _cache.Clear();
+            _metaCache.Clear();
+        }
+
+        /// <summary>
+        /// Tests that caching actually happens.
+        /// </summary>
+        [Test]
+        public void TestResultFromCache()
+        {
+            using (var ctx = GetDbContext())
+            {
+                // Add data.
+                ctx.Posts.Add(new Post {Title = "Foo", Blog = new Blog(), PostId = 1});
+                ctx.Posts.Add(new Post {Title = "Bar", Blog = new Blog(), PostId = 2});
+                ctx.SaveChanges();
+
+                Assert.AreEqual(new[] {"Foo"}, ctx.Posts.Where(x => x.Title == "Foo").Select(x => x.Title).ToArray());
+                Assert.AreEqual(new[] {"Bar"}, ctx.Posts.Where(x => x.Title == "Bar").Select(x => x.Title).ToArray());
+
+                // Alter cached data: swap cached values.
+                
+                var cachedData = _cache.ToArray();
+
+                Assert.AreEqual(2, cachedData.Length);
+
+                _cache[cachedData[0].Key] = cachedData[1].Value;
+                _cache[cachedData[1].Key] = cachedData[0].Value;
+
+                // Verify.
+                Assert.AreEqual(new[] {"Bar"}, ctx.Posts.Where(x => x.Title == "Foo").Select(x => x.Title).ToArray());
+                Assert.AreEqual(new[] {"Foo"}, ctx.Posts.Where(x => x.Title == "Bar").Select(x => x.Title).ToArray());
+            }
+        }
+
+        /// <summary>
+        /// Tests the read-write strategy (default).
+        /// </summary>
+        [Test]
+        public void TestReadWriteStrategy()
+        {
+            using (var ctx = GetDbContext())
+            {
+                var blog = new Blog
+                {
+                    Name = "Foo",
+                    Posts = new List<Post>
+                    {
+                        new Post {Title = "My First Post", Content = "Hello World!"}
+                    }
+                };
+                ctx.Blogs.Add(blog);
+
+                Assert.AreEqual(2, ctx.SaveChanges());
+
+                // Check that query works.
+                Assert.AreEqual(1, ctx.Posts.Where(x => x.Title.StartsWith("My")).ToArray().Length);
+
+                // Add new post to check invalidation.
+                ctx.Posts.Add(new Post {BlogId = blog.BlogId, Title = "My Second Post", Content = "Foo bar."});
+                Assert.AreEqual(1, ctx.SaveChanges());
+
+                Assert.AreEqual(0, _cache.GetSize()); // No cached entries.
+
+                Assert.AreEqual(2, ctx.Posts.Where(x => x.Title.StartsWith("My")).ToArray().Length);
+
+                Assert.AreEqual(1, _cache.GetSize()); // Cached query added.
+
+                // Delete post.
+                ctx.Posts.Remove(ctx.Posts.First());
+                Assert.AreEqual(1, ctx.SaveChanges());
+
+                Assert.AreEqual(0, _cache.GetSize()); // No cached entries.
+                Assert.AreEqual(1, ctx.Posts.Where(x => x.Title.StartsWith("My")).ToArray().Length);
+
+                Assert.AreEqual(1, _cache.GetSize()); // Cached query added.
+
+                // Modify post.
+                Assert.AreEqual(0, ctx.Posts.Count(x => x.Title.EndsWith("updated")));
+
+                ctx.Posts.Single().Title += " - updated";
+                Assert.AreEqual(1, ctx.SaveChanges());
+
+                Assert.AreEqual(0, _cache.GetSize()); // No cached entries.
+                Assert.AreEqual(1, ctx.Posts.Count(x => x.Title.EndsWith("updated")));
+
+                Assert.AreEqual(1, _cache.GetSize()); // Cached query added.
+            }
+        }
+
+        /// <summary>
+        /// Tests the read only strategy.
+        /// </summary>
+        [Test]
+        public void TestReadOnlyStrategy()
+        {
+            // Set up a policy to cache Blogs as read-only and Posts as read-write.
+            Policy.GetCachingStrategyFunc = q =>
+                q.AffectedEntitySets.Count == 1 && q.AffectedEntitySets.Single().Name == "Blog"
+                    ? DbCachingMode.ReadOnly
+                    : DbCachingMode.ReadWrite;
+
+            using (var ctx = GetDbContext())
+            {
+                ctx.Blogs.Add(new Blog
+                {
+                    Name = "Foo",
+                    Posts = new List<Post>
+                    {
+                        new Post {Title = "Post"}
+                    }
+                });
+
+                ctx.SaveChanges();
+
+                // Update entities.
+                Assert.AreEqual("Foo", ctx.Blogs.Single().Name);
+                Assert.AreEqual("Post", ctx.Posts.Single().Title);
+
+                ctx.Blogs.Single().Name += " - updated";
+                ctx.Posts.Single().Title += " - updated";
+
+                ctx.SaveChanges();
+            }
+
+            // Verify that cached result is not changed for blogs, but changed for posts.
+            using (var ctx = GetDbContext())
+            {
+                // Raw SQL queries do not hit cache - verify that actual data is updated.
+                Assert.AreEqual("Foo - updated", ctx.Database.SqlQuery<string>("select name from blogs").Single());
+                Assert.AreEqual("Post - updated", ctx.Database.SqlQuery<string>("select title from posts").Single());
+
+                // Check EF queries that hit cache.
+                Assert.AreEqual("Foo", ctx.Blogs.Single().Name);
+                Assert.AreEqual("Post - updated", ctx.Posts.Single().Title);
+
+            }
+
+            // Clear the cache and verify that actual value in DB is changed.
+            _cache.Clear();
+
+            using (var ctx = GetDbContext())
+            {
+                Assert.AreEqual("Foo - updated", ctx.Blogs.Single().Name);
+                Assert.AreEqual("Post - updated", ctx.Posts.Single().Title);
+            }
+        }
+
+        /// <summary>
+        /// Tests the scalar queries.
+        /// </summary>
+        [Test]
+        public void TestScalars()
+        {
+            using (var ctx = GetDbContext())
+            {
+                var blog = new Blog
+                {
+                    Name = "Foo",
+                    Posts = new List<Post>
+                    {
+                        new Post {Title = "1"},
+                        new Post {Title = "2"},
+                        new Post {Title = "3"},
+                        new Post {Title = "4"}
+                    }
+                };
+                ctx.Blogs.Add(blog);
+
+                Assert.AreEqual(5, ctx.SaveChanges());
+
+                // Test sum and count.
+                const string esql = "SELECT COUNT(1) FROM [BloggingContext].Posts";
+
+                Assert.AreEqual(4, ctx.Posts.Count());
+                Assert.AreEqual(4, ctx.Posts.Count(x => x.Content == null));
+                Assert.AreEqual(4, GetEntityCommand(ctx, esql).ExecuteScalar());
+                Assert.AreEqual(blog.BlogId*4, ctx.Posts.Sum(x => x.BlogId));
+
+                ctx.Posts.Remove(ctx.Posts.First());
+                ctx.SaveChanges();
+
+                Assert.AreEqual(3, ctx.Posts.Count());
+                Assert.AreEqual(3, ctx.Posts.Count(x => x.Content == null));
+                Assert.AreEqual(3, GetEntityCommand(ctx, esql).ExecuteScalar());
+                Assert.AreEqual(blog.BlogId*3, ctx.Posts.Sum(x => x.BlogId));
+            }
+        }
+
+        /// <summary>
+        /// Tests transactions created with BeginTransaction.
+        /// </summary>
+        [Test]
+        public void TestTx()
+        {
+            // Check TX without commit.
+            using (var ctx = GetDbContext())
+            {
+                using (ctx.Database.BeginTransaction())
+                {
+                    ctx.Posts.Add(new Post {Title = "Foo", Blog = new Blog()});
+                    ctx.SaveChanges();
+
+                    Assert.AreEqual(1, ctx.Posts.ToArray().Length);
+                }
+            }
+
+            using (var ctx = GetDbContext())
+            {
+                Assert.AreEqual(0, ctx.Posts.ToArray().Length);
+            }
+
+            // Check TX with commit.
+            using (var ctx = GetDbContext())
+            {
+                using (var tx = ctx.Database.BeginTransaction())
+                {
+                    ctx.Posts.Add(new Post {Title = "Foo", Blog = new Blog()});
+                    ctx.SaveChanges();
+
+                    Assert.AreEqual(1, ctx.Posts.ToArray().Length);
+
+                    tx.Commit();
+
+                    Assert.AreEqual(1, ctx.Posts.ToArray().Length);
+                }
+            }
+
+            using (var ctx = GetDbContext())
+            {
+                Assert.AreEqual(1, ctx.Posts.ToArray().Length);
+            }
+        }
+
+        /// <summary>
+        /// Tests transactions created with TransactionScope.
+        /// </summary>
+        [Test]
+        public void TestTxScope()
+        {
+            // Check TX without commit.
+            using (new TransactionScope())
+            {
+                using (var ctx = GetDbContext())
+                {
+                    ctx.Posts.Add(new Post {Title = "Foo", Blog = new Blog()});
+                    ctx.SaveChanges();
+                }
+            }
+
+            using (var ctx = GetDbContext())
+            {
+                Assert.AreEqual(0, ctx.Posts.ToArray().Length);
+            }
+
+            // Check TX with commit.
+            using (var tx = new TransactionScope())
+            {
+                using (var ctx = GetDbContext())
+                {
+                    ctx.Posts.Add(new Post {Title = "Foo", Blog = new Blog()});
+                    ctx.SaveChanges();
+                }
+
+                tx.Complete();
+            }
+
+            using (var ctx = GetDbContext())
+            {
+                Assert.AreEqual(1, ctx.Posts.ToArray().Length);
+            }
+        }
+
+        /// <summary>
+        /// Tests the expiration.
+        /// </summary>
+        [Test]
+        public void TestExpiration()
+        {
+            Policy.GetExpirationTimeoutFunc = qry => TimeSpan.FromSeconds(0.3);
+
+            using (var ctx = GetDbContext())
+            {
+                ctx.Posts.Add(new Post {Title = "Foo", Blog = new Blog()});
+                ctx.SaveChanges();
+
+                Assert.AreEqual(1, ctx.Posts.ToArray().Length);
+                Assert.AreEqual(1, _cache.GetSize());
+
+                var key = _cache.Single().Key;
+                Assert.IsTrue(_cache.ContainsKey(key));
+
+                Thread.Sleep(300);
+
+                Assert.IsFalse(_cache.ContainsKey(key));
+                Assert.AreEqual(0, _cache.GetSize());
+                Assert.AreEqual(2, _metaCache.GetSize());
+            }
+        }
+
+        /// <summary>
+        /// Tests the caching policy.
+        /// </summary>
+        [Test]
+        public void TestCachingPolicy()
+        {
+            var funcs = new List<string>();
+
+            var checkQry = (Action<DbQueryInfo>) (qry =>
+                {
+                    var set = qry.AffectedEntitySets.Single();
+
+                    Assert.AreEqual("Post", set.Name);
+
+                    Assert.AreEqual(1, qry.Parameters.Count);
+                    Assert.AreEqual(-5, qry.Parameters[0].Value);
+                    Assert.AreEqual(DbType.Int32, qry.Parameters[0].DbType);
+
+                    Assert.IsTrue(qry.CommandText.EndsWith("WHERE [Extent1].[BlogId] > @p__linq__0"));
+                }
+            );
+
+            Policy.CanBeCachedFunc = qry =>
+            {
+                funcs.Add("CanBeCached");
+                checkQry(qry);
+                return true;
+            };
+
+            Policy.CanBeCachedRowsFunc = (qry, rows) =>
+            {
+                funcs.Add("CanBeCachedRows");
+                Assert.AreEqual(3, rows);
+                checkQry(qry);
+                return true;
+            };
+
+            Policy.GetCachingStrategyFunc = qry =>
+            {
+                funcs.Add("GetCachingStrategy");
+                checkQry(qry);
+                return DbCachingMode.ReadWrite;
+            };
+
+            Policy.GetExpirationTimeoutFunc = qry =>
+            {
+                funcs.Add("GetExpirationTimeout");
+                checkQry(qry);
+                return TimeSpan.MaxValue;
+            };
+
+            using (var ctx = GetDbContext())
+            {
+                var blog = new Blog();
+
+                ctx.Posts.Add(new Post {Title = "Foo", Blog = blog});
+                ctx.Posts.Add(new Post {Title = "Bar", Blog = blog});
+                ctx.Posts.Add(new Post {Title = "Baz", Blog = blog});
+
+                ctx.SaveChanges();
+
+                int minId = -5;
+                Assert.AreEqual(3, ctx.Posts.Where(x => x.BlogId > minId).ToArray().Length);
+
+                // Check that policy methods are called in correct order with correct params.
+                Assert.AreEqual(
+                    new[] {"GetCachingStrategy", "CanBeCached", "CanBeCachedRows", "GetExpirationTimeout"},
+                    funcs.ToArray());
+            }
+        }
+
+        /// <summary>
+        /// Tests the cache reader indirectly with an entity that has various field types.
+        /// </summary>
+        [Test]
+        public void TestCacheReader()
+        {
+            // Tests all kinds of entity field types to cover ArrayDbDataReader.
+            var test = GetTestEntity();
+
+            using (var ctx = new BloggingContext(ConnectionString))
+            {
+                ctx.Tests.Add(test);
+                ctx.SaveChanges();
+            }
+
+            // Use new context to ensure no first-level caching.
+            using (var ctx = new BloggingContext(ConnectionString))
+            {
+                // Check default deserialization.
+                var test0 = ctx.Tests.Single(x => x.Bool);
+                Assert.AreEqual(test, test0);
+            }
+        }
+
+        /// <summary>
+        /// Tests the cache reader by calling it directly.
+        /// These calls are (partly) delegated by EF to the <see cref="ArrayDbDataReader"/>.
+        /// </summary>
+        [Test]
+        public void TestCacheReaderRaw()
+        {
+            var test = GetTestEntity();
+
+            using (var ctx = new BloggingContext(ConnectionString))
+            {
+                ctx.Tests.Add(test);
+                ctx.SaveChanges();
+
+                test = ctx.Tests.Single();
+            }
+
+            using (var ctx = new BloggingContext(ConnectionString))
+            {
+                var cmd = GetEntityCommand(ctx, "SELECT VALUE Test FROM BloggingContext.Tests AS Test");
+
+                using (var reader = cmd.ExecuteReader(CommandBehavior.SequentialAccess))
+                {
+                    // Check schema.
+                    Assert.Throws<NotSupportedException>(() => reader.GetSchemaTable());
+                    Assert.AreEqual(0, reader.Depth);
+                    Assert.AreEqual(-1, reader.RecordsAffected);
+                    Assert.IsTrue(reader.HasRows);
+                    Assert.IsFalse(reader.IsClosed);
+                    Assert.AreEqual(11, reader.FieldCount);
+                    Assert.AreEqual(11, reader.VisibleFieldCount);
+
+                    // Check field names.
+                    Assert.AreEqual("Edm.Int32", reader.GetDataTypeName(0));
+                    Assert.AreEqual("Edm.Byte", reader.GetDataTypeName(1));
+                    Assert.AreEqual("Edm.Int16", reader.GetDataTypeName(2));
+                    Assert.AreEqual("Edm.Int64", reader.GetDataTypeName(3));
+                    Assert.AreEqual("Edm.Single", reader.GetDataTypeName(4));
+                    Assert.AreEqual("Edm.Double", reader.GetDataTypeName(5));
+                    Assert.AreEqual("Edm.Decimal", reader.GetDataTypeName(6));
+                    Assert.AreEqual("Edm.Boolean", reader.GetDataTypeName(7));
+                    Assert.AreEqual("Edm.String", reader.GetDataTypeName(8));
+                    Assert.AreEqual("Edm.Guid", reader.GetDataTypeName(9));
+                    Assert.AreEqual("Edm.DateTime", reader.GetDataTypeName(10));
+
+                    // Check field types.
+                    Assert.AreEqual(typeof(int), reader.GetFieldType(0));
+                    Assert.AreEqual(typeof(byte), reader.GetFieldType(1));
+                    Assert.AreEqual(typeof(short), reader.GetFieldType(2));
+                    Assert.AreEqual(typeof(long), reader.GetFieldType(3));
+                    Assert.AreEqual(typeof(float), reader.GetFieldType(4));
+                    Assert.AreEqual(typeof(double), reader.GetFieldType(5));
+                    Assert.AreEqual(typeof(decimal), reader.GetFieldType(6));
+                    Assert.AreEqual(typeof(bool), reader.GetFieldType(7));
+                    Assert.AreEqual(typeof(string), reader.GetFieldType(8));
+                    Assert.AreEqual(typeof(Guid), reader.GetFieldType(9));
+                    Assert.AreEqual(typeof(DateTime), reader.GetFieldType(10));
+
+                    // Read.
+                    Assert.IsTrue(reader.Read());
+
+                    // Test values array.
+                    var vals = new object[reader.FieldCount];
+                    reader.GetValues(vals);
+
+                    Assert.AreEqual(test.Byte, vals[reader.GetOrdinal("Byte")]);
+                    Assert.AreEqual(test.Short, vals[reader.GetOrdinal("Short")]);
+                    Assert.AreEqual(test.ArrayReaderTestId, vals[reader.GetOrdinal("ArrayReaderTestId")]);
+                    Assert.AreEqual(test.Long, vals[reader.GetOrdinal("Long")]);
+                    Assert.AreEqual(test.Float, vals[reader.GetOrdinal("Float")]);
+                    Assert.AreEqual(test.Double, vals[reader.GetOrdinal("Double")]);
+                    Assert.AreEqual(test.Decimal, vals[reader.GetOrdinal("Decimal")]);
+                    Assert.AreEqual(test.Bool, vals[reader.GetOrdinal("Bool")]);
+                    Assert.AreEqual(test.String, vals[reader.GetOrdinal("String")]);
+                    Assert.AreEqual(test.Guid, vals[reader.GetOrdinal("Guid")]);
+                    Assert.AreEqual(test.DateTime, vals[reader.GetOrdinal("DateTime")]);
+                }
+
+                using (var reader = cmd.ExecuteReader(CommandBehavior.SequentialAccess))
+                {
+                    // Read.
+                    Assert.IsTrue(reader.Read());
+
+                    // Test separate values.
+                    Assert.AreEqual(test.ArrayReaderTestId, reader.GetInt32(0));
+                    Assert.AreEqual(test.Byte, reader.GetByte(1));
+                    Assert.AreEqual(test.Short, reader.GetInt16(2));
+                    Assert.AreEqual(test.Long, reader.GetInt64(3));
+                    Assert.AreEqual(test.Float, reader.GetFloat(4));
+                    Assert.AreEqual(test.Double, reader.GetDouble(5));
+                    Assert.AreEqual(test.Decimal, reader.GetDecimal(6));
+                    Assert.AreEqual(test.Bool, reader.GetBoolean(7));
+                    Assert.AreEqual(test.String, reader.GetString(8));
+                    Assert.AreEqual(test.Guid, reader.GetGuid(9));
+                    Assert.AreEqual(test.DateTime, reader.GetDateTime(10));
+                }
+            }
+        }
+
+        /// <summary>
+        /// Tests the database context.
+        /// </summary>
+        [Test]
+        public void TestDbContext()
+        {
+            using (var ctx = GetDbContext())
+            {
+                var objCtx = ((IObjectContextAdapter) ctx).ObjectContext;
+
+                var script = objCtx.CreateDatabaseScript();
+                Assert.IsTrue(script.StartsWith("CREATE TABLE \"Blogs\""));
+            }
+        }
+
+        /// <summary>
+        /// Tests that old versions of caches entries are cleaned up.
+        /// </summary>
+        [Test]
+        public void TestOldEntriesCleanup()
+        {
+            // Run in a loop to generate a bunch of outdated cache entries.
+            for (var i = 0; i < 100; i++)
+                CreateRemoveBlog();
+
+            // Only one version of data is in the cache.
+            Assert.AreEqual(1, _cache.GetSize());
+            Assert.AreEqual(1, _metaCache.GetSize());
+        }
+
+        /// <summary>
+        /// Tests the old entries cleanup in multi threaded scenario.
+        /// </summary>
+        [Test]
+        [Category(TestUtils.CategoryIntensive)]
+        public void TestOldEntriesCleanupMultithreaded()
+        {
+            TestUtils.RunMultiThreaded(CreateRemoveBlog, 4, 20);
+
+            // Wait for the cleanup to complete.
+            Thread.Sleep(200);
+
+            // Only one version of data is in the cache.
+            Assert.AreEqual(1, _cache.GetSize());
+            Assert.AreEqual(1, _metaCache.GetSize());
+        }
+
+        /// <summary>
+        /// Tests the entity set version increment in multi-threaded scenario.
+        /// </summary>
+        [Test]
+        [Category(TestUtils.CategoryIntensive)]
+        public void TestIncrementMultithreaded()
+        {
+            var opCnt = 0;
+
+            TestUtils.RunMultiThreaded(() =>
+            {
+                var blog = new Blog {Name = "my blog"};
+                using (var ctx = GetDbContext())
+                {
+                    ctx.Blogs.Add(blog);
+                    ctx.SaveChanges();
+                }
+
+                Interlocked.Increment(ref opCnt);
+
+                using (var ctx = GetDbContext())
+                {
+                    ctx.Blogs.Attach(blog);
+                    ctx.Blogs.Remove(blog);
+                    ctx.SaveChanges();
+                }
+
+                Interlocked.Increment(ref opCnt);
+            }, 4, 10);
+
+            var setVersion = _metaCache["Blog"];
+
+            Assert.AreEqual(opCnt, setVersion);
+        }
+
+        /// <summary>
+        /// Creates and removes a blog.
+        /// </summary>
+        private void CreateRemoveBlog()
+        {
+            try
+            {
+                CreateRemoveBlog0();
+            }
+            catch (Exception ex)
+            {
+                // Ignore SQL CE glitch.
+                if (!ex.ToString().Contains("The current row was deleted."))
+                    throw;
+            }
+        }
+
+        /// <summary>
+        /// Creates and removes a blog.
+        /// </summary>
+        private void CreateRemoveBlog0()
+        {
+            var blog = new Blog {Name = "my blog"};
+            var threadId = Thread.CurrentThread.ManagedThreadId;
+
+            Func<object> getMeta = () => _metaCache.Where(x => x.Key.Equals("Blog"))
+                .Select(x => x.Value).SingleOrDefault() ?? "null";
+
+            var meta1 = getMeta();
+
+            using (var ctx = GetDbContext())
+            {
+                ctx.Blogs.Add(blog);
+                ctx.SaveChanges();
+            }
+
+            var meta2 = getMeta();
+
+            using (var ctx = GetDbContext())
+            {
+                // Use ToArray so that there is always the same DB query.
+                Assert.AreEqual(1, ctx.Blogs.ToArray().Count(x => x.BlogId == blog.BlogId),
+                    string.Format("Existing blog not found: {0} = {1}, {2} | {3}", blog.BlogId, meta1, meta2, 
+                    threadId));
+            }
+
+            var meta3 = getMeta();
+
+            using (var ctx = GetDbContext())
+            {
+                ctx.Blogs.Attach(blog);
+                ctx.Blogs.Remove(blog);
+                ctx.SaveChanges();
+            }
+
+            var meta4 = getMeta();
+
+            using (var ctx = GetDbContext())
+            {
+                // Use ToArray so that there is always the same DB query.
+                Assert.AreEqual(0, ctx.Blogs.ToArray().Count(x => x.BlogId == blog.BlogId),
+                    string.Format("Found removed blog: {0} = {1}, {2}, {3}, {4} | {5}", blog.BlogId, meta1, 
+                    meta2, meta3, meta4, threadId));
+            }
+        }
+
+        /// <summary>
+        /// Executes the entity SQL.
+        /// </summary>
+        private static EntityCommand GetEntityCommand(IObjectContextAdapter ctx, string esql)
+        {
+            var objCtx = ctx.ObjectContext;
+
+            var conn = objCtx.Connection;
+            conn.Open();
+
+            var cmd = (EntityCommand) conn.CreateCommand();
+            cmd.CommandText = esql;
+
+            return cmd;
+        }
+
+        /// <summary>
+        /// Gets the test entity.
+        /// </summary>
+        private static ArrayReaderTest GetTestEntity()
+        {
+            return new ArrayReaderTest
+            {
+                DateTime = DateTime.Today,
+                Bool = true,
+                Byte = 56,
+                String = "z",
+                Decimal = (decimal)5.6,
+                Double = 7.8d,
+                Float = -4.5f,
+                Guid = Guid.NewGuid(),
+                ArrayReaderTestId = -8,
+                Long = 3,
+                Short = 5
+            };
+        }
+
+        /// <summary>
+        /// Gets the database context.
+        /// </summary>
+        private static BloggingContext GetDbContext()
+        {
+            return new BloggingContext(ConnectionString);
+        }
+
+        private class MyDbConfiguration : IgniteDbConfiguration
+        {
+            public MyDbConfiguration() : base(Ignition.GetIgnite(), null, null, Policy)
+            {
+                // No-op.
+            }
+        }
+
+        [DbConfigurationType(typeof(MyDbConfiguration))]
+        private class BloggingContext : DbContext
+        {
+            public BloggingContext(string nameOrConnectionString) : base(nameOrConnectionString)
+            {
+                // No-op.
+            }
+
+            public virtual DbSet<Blog> Blogs { get; set; }
+            public virtual DbSet<Post> Posts { get; set; }
+            public virtual DbSet<ArrayReaderTest> Tests { get; set; }
+        }
+
+        private class Blog
+        {
+            public int BlogId { get; set; }
+            public string Name { get; set; }
+
+            public virtual List<Post> Posts { get; set; }
+        }
+
+        private class Post
+        {
+            public int PostId { get; set; }
+            public string Title { get; set; }
+            public string Content { get; set; }
+
+            public int BlogId { get; set; }
+            public virtual Blog Blog { get; set; }
+        }
+
+        private class ArrayReaderTest
+        {
+            public byte Byte { get; set; }
+            public short Short { get; set; }
+            public int ArrayReaderTestId { get; set; }
+            public long Long { get; set; }
+            public float Float { get; set; }
+            public double Double { get; set; }
+            public decimal Decimal { get; set; }
+            public bool Bool { get; set; }
+            public string String { get; set; }
+            public Guid Guid { get; set; }
+            public DateTime DateTime { get; set; }
+
+            private bool Equals(ArrayReaderTest other)
+            {
+                return Byte == other.Byte && Short == other.Short &&
+                       ArrayReaderTestId == other.ArrayReaderTestId && Long == other.Long && 
+                       Float.Equals(other.Float) && Double.Equals(other.Double) && 
+                       Decimal == other.Decimal && Bool == other.Bool && String == other.String && 
+                       Guid.Equals(other.Guid) && DateTime.Equals(other.DateTime);
+            }
+
+            public override bool Equals(object obj)
+            {
+                if (ReferenceEquals(null, obj)) return false;
+                if (ReferenceEquals(this, obj)) return true;
+                if (obj.GetType() != GetType()) return false;
+                return Equals((ArrayReaderTest) obj);
+            }
+
+            public override int GetHashCode()
+            {
+                unchecked
+                {
+                    var hashCode = Byte.GetHashCode();
+                    hashCode = (hashCode*397) ^ Short.GetHashCode();
+                    hashCode = (hashCode*397) ^ ArrayReaderTestId;
+                    hashCode = (hashCode*397) ^ Long.GetHashCode();
+                    hashCode = (hashCode*397) ^ Float.GetHashCode();
+                    hashCode = (hashCode*397) ^ Double.GetHashCode();
+                    hashCode = (hashCode*397) ^ Decimal.GetHashCode();
+                    hashCode = (hashCode*397) ^ Bool.GetHashCode();
+                    hashCode = (hashCode*397) ^ String.GetHashCode();
+                    hashCode = (hashCode*397) ^ Guid.GetHashCode();
+                    hashCode = (hashCode*397) ^ DateTime.GetHashCode();
+                    return hashCode;
+                }
+            }
+        }
+
+        private class DelegateCachingPolicy : IDbCachingPolicy
+        {
+            public Func<DbQueryInfo, bool> CanBeCachedFunc { get; set; }
+
+            public Func<DbQueryInfo, int, bool> CanBeCachedRowsFunc { get; set; }
+
+            public Func<DbQueryInfo, TimeSpan> GetExpirationTimeoutFunc { get; set; }
+
+            public Func<DbQueryInfo, DbCachingMode> GetCachingStrategyFunc { get; set; }
+
+            public bool CanBeCached(DbQueryInfo queryInfo)
+            {
+                return CanBeCachedFunc == null || CanBeCachedFunc(queryInfo);
+            }
+
+            public bool CanBeCached(DbQueryInfo queryInfo, int rowCount)
+            {
+                return CanBeCachedRowsFunc == null || CanBeCachedRowsFunc(queryInfo, rowCount);
+            }
+
+            public TimeSpan GetExpirationTimeout(DbQueryInfo queryInfo)
+            {
+                return GetExpirationTimeoutFunc == null ? TimeSpan.MaxValue : GetExpirationTimeoutFunc(queryInfo);
+            }
+
+            public DbCachingMode GetCachingMode(DbQueryInfo queryInfo)
+            {
+                return GetCachingStrategyFunc == null ? DbCachingMode.ReadWrite : GetCachingStrategyFunc(queryInfo);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Properties/AssemblyInfo.cs
new file mode 100644
index 0000000..fe5e7ce
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Properties/AssemblyInfo.cs
@@ -0,0 +1,39 @@
+\ufeff/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*      http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+using System;
+using System.Reflection;
+using System.Runtime.InteropServices;
+
+[assembly: AssemblyTitle("Apache.Ignite.EntityFramework.Tests")]
+[assembly: AssemblyDescription("")]
+[assembly: AssemblyConfiguration("")]
+[assembly: AssemblyCompany("Apache Software Foundation")]
+[assembly: AssemblyProduct("Apache Ignite.NET")]
+[assembly: AssemblyCopyright("Copyright �  2015")]
+[assembly: AssemblyTrademark("")]
+[assembly: AssemblyCulture("")]
+
+[assembly: ComVisible(false)]
+
+[assembly: Guid("cda5700e-78f3-4a9e-a9b0-704cbe94651c")]
+
+[assembly: AssemblyVersion("1.8.0.14218")]
+[assembly: AssemblyFileVersion("1.8.0.14218")]
+[assembly: AssemblyInformationalVersion("1.8.0")]
+
+[assembly: CLSCompliant(true)]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/packages.config
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/packages.config b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/packages.config
new file mode 100644
index 0000000..42a3b73
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/packages.config
@@ -0,0 +1,23 @@
+\ufeff<?xml version="1.0" encoding="utf-8"?>
+<!--
+  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.
+-->
+<packages>
+  <package id="NUnit.Runners" version="2.6.3" targetFramework="net40" />
+  <package id="EntityFramework" version="6.1.3" targetFramework="net40" />
+  <package id="EntityFramework.SqlServerCompact" version="6.1.3" targetFramework="net40" />
+  <package id="Microsoft.SqlServer.Compact" version="4.0.8876.1" targetFramework="net40" />
+</packages>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.csproj b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.csproj
new file mode 100644
index 0000000..8b3c651
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.csproj
@@ -0,0 +1,93 @@
+\ufeff<?xml version="1.0" encoding="utf-8"?>
+<Project ToolsVersion="4.0" DefaultTargets="Build" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
+  <Import Project="$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props" Condition="Exists('$(MSBuildExtensionsPath)\$(MSBuildToolsVersion)\Microsoft.Common.props')" />
+  <PropertyGroup>
+    <Configuration Condition=" '$(Configuration)' == '' ">Debug</Configuration>
+    <Platform Condition=" '$(Platform)' == '' ">AnyCPU</Platform>
+    <ProjectGuid>{C558518A-C1A0-4224-AAA9-A8688474B4DC}</ProjectGuid>
+    <OutputType>Library</OutputType>
+    <AppDesignerFolder>Properties</AppDesignerFolder>
+    <RootNamespace>Apache.Ignite.EntityFramework</RootNamespace>
+    <AssemblyName>Apache.Ignite.EntityFramework</AssemblyName>
+    <TargetFrameworkVersion>v4.0</TargetFrameworkVersion>
+    <FileAlignment>512</FileAlignment>
+  </PropertyGroup>
+  <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Debug|AnyCPU' ">
+    <DebugSymbols>true</DebugSymbols>
+    <DebugType>full</DebugType>
+    <Optimize>false</Optimize>
+    <OutputPath>bin\Debug\</OutputPath>
+    <DefineConstants>DEBUG;TRACE</DefineConstants>
+    <ErrorReport>prompt</ErrorReport>
+    <WarningLevel>4</WarningLevel>
+    <RunCodeAnalysis>true</RunCodeAnalysis>
+    <CodeAnalysisRuleSet>AllRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
+  </PropertyGroup>
+  <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release|AnyCPU' ">
+    <DebugType>pdbonly</DebugType>
+    <Optimize>true</Optimize>
+    <OutputPath>bin\Release\</OutputPath>
+    <DefineConstants>TRACE</DefineConstants>
+    <ErrorReport>prompt</ErrorReport>
+    <WarningLevel>4</WarningLevel>
+  </PropertyGroup>
+  <PropertyGroup>
+    <SignAssembly>true</SignAssembly>
+  </PropertyGroup>
+  <PropertyGroup>
+    <AssemblyOriginatorKeyFile>Apache.Ignite.EntityFramework.snk</AssemblyOriginatorKeyFile>
+  </PropertyGroup>
+  <ItemGroup>
+    <Reference Include="EntityFramework, Version=6.0.0.0, Culture=neutral, PublicKeyToken=b77a5c561934e089, processorArchitecture=MSIL">
+      <HintPath>..\packages\EntityFramework.6.1.3\lib\net40\EntityFramework.dll</HintPath>
+      <Private>True</Private>
+    </Reference>
+    <Reference Include="System" />
+    <Reference Include="System.Configuration" />
+    <Reference Include="System.Core" />
+    <Reference Include="System.Data" />
+    <Reference Include="System.Xml" />
+  </ItemGroup>
+  <ItemGroup>
+    <Compile Include="DbCachingMode.cs" />
+    <Compile Include="DbQueryInfo.cs" />
+    <Compile Include="IDbCachingPolicy.cs" />
+    <Compile Include="Impl\ArrayDbDataReader.cs" />
+    <Compile Include="Impl\DataReaderResult.cs" />
+    <Compile Include="Impl\DbCacheKey.cs" />
+    <Compile Include="Impl\DbCommandDefinitionProxy.cs" />
+    <Compile Include="Impl\DbCommandInfo.cs" />
+    <Compile Include="Impl\DbCommandProxy.cs">
+      <SubType>Component</SubType>
+    </Compile>
+    <Compile Include="Impl\DbProviderServicesProxy.cs" />
+    <Compile Include="Impl\DataReaderField.cs" />
+    <Compile Include="DbCachingPolicy.cs" />
+    <Compile Include="IgniteDbConfiguration.cs" />
+    <Compile Include="Impl\DbCache.cs" />
+    <Compile Include="Impl\DbTransactionInterceptor.cs" />
+    <Compile Include="Properties\AssemblyInfo.cs" />
+  </ItemGroup>
+  <ItemGroup>
+    <None Include="Apache.Ignite.EntityFramework.snk" />
+    <None Include="packages.config" />
+  </ItemGroup>
+  <ItemGroup>
+    <ProjectReference Include="..\Apache.Ignite.Core\Apache.Ignite.Core.csproj">
+      <Project>{4CD2F726-7E2B-46C4-A5BA-057BB82EECB6}</Project>
+      <Name>Apache.Ignite.Core</Name>
+    </ProjectReference>
+  </ItemGroup>
+  <ItemGroup>
+    <None Include="Apache.Ignite.EntityFramework.nuspec" />
+  </ItemGroup>
+  <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
+  <!-- To modify your build process, add your task inside one of the targets below and uncomment it. 
+       Other similar extension points exist, see Microsoft.Common.targets.
+  <Target Name="BeforeBuild">
+  </Target>
+  <Target Name="AfterBuild">
+  </Target>
+  -->
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.nuspec
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.nuspec b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.nuspec
new file mode 100644
index 0000000..b8bcd46
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.nuspec
@@ -0,0 +1,57 @@
+\ufeff<?xml version="1.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.
+-->
+
+<!-- 
+
+Creating NuGet package:
+1) Build Apache.Ignite.sln (AnyCPU configuration)
+2) Create package (use csproj instead of nuspec so that template substitution works): 
+   nuget pack Apache.Ignite.EntityFramework.csproj -Prop Configuration=Release -Prop Platform=AnyCPU
+
+-->
+
+<package >
+    <metadata>
+        <id>Apache.Ignite.EntityFramework</id>
+        <title>Apache Ignite Entity Framework Integration</title>
+        <!-- -->
+        <version>$version$</version>
+        <authors>Apache Ignite</authors>
+        <owners>Apache Software Foundation</owners>
+        <licenseUrl>http://www.apache.org/licenses/LICENSE-2.0</licenseUrl>
+        <projectUrl>https://ignite.apache.org/</projectUrl>
+        <iconUrl>https://ignite.apache.org/images/logo_ignite_32_32.png</iconUrl>
+        <requireLicenseAcceptance>false</requireLicenseAcceptance>
+        <description>
+Apache Ignite EntityFramework Second Level Cache: caches EF query results in a distributed in-memory cache.
+            
+More info: https://apacheignite-net.readme.io/
+        </description>
+        <summary>
+            Apache Ignite EntityFramework Integration
+        </summary>
+        <releaseNotes></releaseNotes>
+        <copyright>Copyright 2016</copyright>
+        <tags>EntityFramework Second-Level Apache Ignite In-Memory Distributed Computing SQL NoSQL Grid Map Reduce Cache</tags>
+        <dependencies>
+            <dependency id="Apache.Ignite" version="[$version$]" />
+            <dependency id="EntityFramework" version="[6.1.0,7.0.0)" />
+        </dependencies>    
+    </metadata>
+</package>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.snk
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.snk b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.snk
new file mode 100644
index 0000000..799e742
Binary files /dev/null and b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.snk differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/DbCachingMode.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/DbCachingMode.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/DbCachingMode.cs
new file mode 100644
index 0000000..b38400c
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/DbCachingMode.cs
@@ -0,0 +1,48 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.EntityFramework
+{
+    using System.Data.Entity;
+
+    /// <summary>
+    /// Represents a second-level caching strategy.
+    /// </summary>
+    public enum DbCachingMode
+    {
+        /// <summary>
+        /// Read-only mode, never invalidates.
+        /// <para />
+        /// Database updates are ignored in this mode. Once query results have been cached, they are kept in cache 
+        /// until expired (forever when no expiration is specified).
+        /// <para />
+        /// This mode is suitable for data that is not expected to change 
+        /// (like a list of countries and other dictionary data).
+        /// </summary>
+        ReadOnly,
+
+        /// <summary>
+        /// Read-write mode. Cached data is invalidated when underlying entity set changes.
+        /// <para />
+        /// This is "normal" cache mode which always provides correct query results.
+        /// <para />
+        /// Keep in mind that this mode works correctly only when all database changes are performed 
+        /// via <see cref="DbContext"/> with Ignite caching configured. Other database updates are not tracked.
+        /// </summary>
+        ReadWrite
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/DbCachingPolicy.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/DbCachingPolicy.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/DbCachingPolicy.cs
new file mode 100644
index 0000000..9e05ca9
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/DbCachingPolicy.cs
@@ -0,0 +1,71 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.EntityFramework
+{
+    using System;
+
+    /// <summary>
+    /// Default caching policy implementation: everything is cached with <see cref="DbCachingMode.ReadWrite"/>, 
+    /// no expiration.
+    /// </summary>
+    public class DbCachingPolicy : IDbCachingPolicy
+    {
+        /// <summary>
+        /// Determines whether the specified query can be cached.
+        /// </summary>
+        /// <param name="queryInfo">The query information.</param>
+        /// <returns>
+        ///   <c>true</c> if the specified query can be cached; otherwise, <c>false</c>.
+        /// </returns>
+        public virtual bool CanBeCached(DbQueryInfo queryInfo)
+        {
+            return true;
+        }
+
+        /// <summary>
+        /// Determines whether specified number of rows should be cached.
+        /// </summary>
+        /// <param name="queryInfo">The query information.</param>
+        /// <param name="rowCount">The count of fetched rows.</param>
+        /// <returns></returns>
+        public virtual bool CanBeCached(DbQueryInfo queryInfo, int rowCount)
+        {
+            return true;
+        }
+
+        /// <summary>
+        /// Gets the absolute expiration timeout for a given query.
+        /// </summary>
+        /// <param name="queryInfo">The query information.</param>
+        /// <returns>Expiration timeout. <see cref="TimeSpan.MaxValue"/> for no expiration.</returns>
+        public virtual TimeSpan GetExpirationTimeout(DbQueryInfo queryInfo)
+        {
+            return TimeSpan.MaxValue;
+        }
+
+        /// <summary>
+        /// Gets the caching strategy for a give query.
+        /// </summary>
+        /// <param name="queryInfo">The query information.</param>
+        /// <returns>Caching strategy for the query.</returns>
+        public virtual DbCachingMode GetCachingMode(DbQueryInfo queryInfo)
+        {
+            return DbCachingMode.ReadWrite;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/DbQueryInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/DbQueryInfo.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/DbQueryInfo.cs
new file mode 100644
index 0000000..5ec5446
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/DbQueryInfo.cs
@@ -0,0 +1,78 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.EntityFramework
+{
+    using System.Collections.Generic;
+    using System.Data.Common;
+    using System.Data.Entity.Core.Metadata.Edm;
+    using System.Diagnostics;
+
+    /// <summary>
+    /// Query info.
+    /// </summary>
+    public class DbQueryInfo
+    {
+        /** */
+        private readonly ICollection<EntitySetBase> _affectedEntitySets;
+
+        /** */
+        private readonly string _commandText;
+        
+        /** */
+        private readonly DbParameterCollection _parameters;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="DbQueryInfo"/> class.
+        /// </summary>
+        internal DbQueryInfo(ICollection<EntitySetBase> affectedEntitySets, string commandText, 
+            DbParameterCollection parameters)
+        {
+            Debug.Assert(affectedEntitySets != null);
+            Debug.Assert(commandText != null);
+            Debug.Assert(parameters != null);
+
+            _affectedEntitySets = affectedEntitySets;
+            _commandText = commandText;
+            _parameters = parameters;
+        }
+
+        /// <summary>
+        /// Gets the affected entity sets.
+        /// </summary>
+        public ICollection<EntitySetBase> AffectedEntitySets
+        {
+            get { return _affectedEntitySets; }
+        }
+
+        /// <summary>
+        /// Gets the command text.
+        /// </summary>
+        public string CommandText
+        {
+            get { return _commandText; }
+        }
+
+        /// <summary>
+        /// Gets the parameters.
+        /// </summary>
+        public DbParameterCollection Parameters
+        {
+            get { return _parameters; }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/IDbCachingPolicy.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/IDbCachingPolicy.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/IDbCachingPolicy.cs
new file mode 100644
index 0000000..504ab5e
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/IDbCachingPolicy.cs
@@ -0,0 +1,58 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.EntityFramework
+{
+    using System;
+
+    /// <summary>
+    /// Caching policy: defines which queries should be cached.
+    /// </summary>
+    public interface IDbCachingPolicy
+    {
+        /// <summary>
+        /// Determines whether the specified query can be cached.
+        /// </summary>
+        /// <param name="queryInfo">The query information.</param>
+        /// <returns>
+        ///   <c>true</c> if the specified query can be cached; otherwise, <c>false</c>.
+        /// </returns>
+        bool CanBeCached(DbQueryInfo queryInfo);
+
+        /// <summary>
+        /// Determines whether specified number of rows should be cached.
+        /// </summary>
+        /// <param name="queryInfo">The query information.</param>
+        /// <param name="rowCount">The count of fetched rows.</param>
+        /// <returns></returns>
+        bool CanBeCached(DbQueryInfo queryInfo, int rowCount);
+
+        /// <summary>
+        /// Gets the absolute expiration timeout for a given query.
+        /// </summary>
+        /// <param name="queryInfo">The query information.</param>
+        /// <returns>Expiration timeout. <see cref="TimeSpan.MaxValue"/> for no expiration.</returns>
+        TimeSpan GetExpirationTimeout(DbQueryInfo queryInfo);
+
+        /// <summary>
+        /// Gets the caching strategy for a give query.
+        /// </summary>
+        /// <param name="queryInfo">The query information.</param>
+        /// <returns>Caching strategy for the query.</returns>
+        DbCachingMode GetCachingMode(DbQueryInfo queryInfo);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/IgniteDbConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/IgniteDbConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/IgniteDbConfiguration.cs
new file mode 100644
index 0000000..c467f94
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/IgniteDbConfiguration.cs
@@ -0,0 +1,240 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.EntityFramework
+{
+    using System.Configuration;
+    using System.Data.Entity;
+    using System.Data.Entity.Core.Common;
+    using System.Diagnostics.CodeAnalysis;
+    using System.Globalization;
+    using Apache.Ignite.Core;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Common;
+    using Apache.Ignite.EntityFramework.Impl;
+
+    /// <summary>
+    /// <see cref="DbConfiguration"/> implementation that uses Ignite as a second-level cache 
+    /// for Entity Framework queries.
+    /// </summary>
+    public class IgniteDbConfiguration : DbConfiguration
+    {
+        /// <summary>
+        /// The configuration section name to be used when starting Ignite.
+        /// </summary>
+        private const string ConfigurationSectionName = "igniteConfiguration";
+
+        /// <summary>
+        /// The default cache name to be used for cached EF data.
+        /// </summary>
+        public const string DefaultCacheNamePrefix = "entityFrameworkQueryCache";
+
+        /// <summary>
+        /// Suffix for the meta cache name.
+        /// </summary>
+        private const string MetaCacheSuffix = "_metadata";
+
+        /// <summary>
+        /// Suffix for the data cache name.
+        /// </summary>
+        private const string DataCacheSuffix = "_data";
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteDbConfiguration"/> class.
+        /// <para />
+        /// This constructor uses default Ignite instance (with null <see cref="IgniteConfiguration.GridName"/>) 
+        /// and a cache with <see cref="DefaultCacheNamePrefix"/> name.
+        /// <para />
+        /// Ignite instance will be started automatically, if it is not started yet.
+        /// <para /> 
+        /// <see cref="IgniteConfigurationSection"/> with name 
+        /// <see cref="ConfigurationSectionName"/> will be picked up when starting Ignite, if present.
+        /// </summary>
+        public IgniteDbConfiguration() 
+            : this(GetConfiguration(ConfigurationSectionName, false), 
+                  GetDefaultMetaCacheConfiguration(DefaultCacheNamePrefix), 
+                  GetDefaultDataCacheConfiguration(DefaultCacheNamePrefix), null)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteDbConfiguration" /> class.
+        /// </summary>
+        /// <param name="configurationSectionName">Name of the configuration section.</param>
+        /// <param name="cacheNamePrefix">The cache name prefix for Data and Metadata caches.</param>
+        /// <param name="policy">The caching policy. Null for default <see cref="DbCachingPolicy" />.</param>
+        public IgniteDbConfiguration(string configurationSectionName, string cacheNamePrefix, IDbCachingPolicy policy)
+            : this(configurationSectionName,
+                GetDefaultMetaCacheConfiguration(cacheNamePrefix),
+                GetDefaultDataCacheConfiguration(cacheNamePrefix), policy)
+
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteDbConfiguration"/> class.
+        /// </summary>
+        /// <param name="configurationSectionName">Name of the configuration section.</param>
+        /// <param name="metaCacheConfiguration">
+        /// Configuration of the metadata cache which holds entity set information. Null for default configuration.
+        /// <para />
+        /// This cache holds small amount of data, but should not lose entries. At least one backup recommended.
+        /// </param>
+        /// <param name="dataCacheConfiguration">
+        /// Configuration of the data cache which holds query results. Null for default configuration.
+        /// <para />
+        /// This cache tolerates lost data and can have no backups.
+        /// </param>
+        /// <param name="policy">The caching policy. Null for default <see cref="DbCachingPolicy"/>.</param>
+        public IgniteDbConfiguration(string configurationSectionName, CacheConfiguration metaCacheConfiguration,
+            CacheConfiguration dataCacheConfiguration, IDbCachingPolicy policy)
+            : this(GetConfiguration(configurationSectionName, true), 
+                  metaCacheConfiguration, dataCacheConfiguration, policy)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteDbConfiguration" /> class.
+        /// </summary>
+        /// <param name="igniteConfiguration">The ignite configuration to use for starting Ignite instance.</param>
+        /// <param name="metaCacheConfiguration">
+        /// Configuration of the metadata cache which holds entity set information. Null for default configuration. 
+        /// <para />
+        /// This cache holds small amount of data, but should not lose entries. At least one backup recommended.
+        /// </param>
+        /// <param name="dataCacheConfiguration">
+        /// Configuration of the data cache which holds query results. Null for default configuration.
+        /// <para />
+        /// This cache tolerates lost data and can have no backups.
+        /// </param>
+        /// <param name="policy">The caching policy. Null for default <see cref="DbCachingPolicy"/>.</param>
+        public IgniteDbConfiguration(IgniteConfiguration igniteConfiguration,
+            CacheConfiguration metaCacheConfiguration, CacheConfiguration dataCacheConfiguration,
+            IDbCachingPolicy policy)
+            : this(GetOrStartIgnite(igniteConfiguration), metaCacheConfiguration, dataCacheConfiguration, policy)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="IgniteDbConfiguration" /> class.
+        /// </summary>
+        /// <param name="ignite">The ignite instance to use.</param>
+        /// <param name="metaCacheConfiguration">
+        /// Configuration of the metadata cache which holds entity set information. Null for default configuration. 
+        /// <para />
+        /// This cache holds small amount of data, but should not lose entries. At least one backup recommended.
+        /// </param>
+        /// <param name="dataCacheConfiguration">
+        /// Configuration of the data cache which holds query results. Null for default configuration.
+        /// <para />
+        /// This cache tolerates lost data and can have no backups.
+        /// </param>
+        /// <param name="policy">The caching policy. Null for default <see cref="DbCachingPolicy" />.</param>
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods", 
+            Justification = "Validation is present")]
+        public IgniteDbConfiguration(IIgnite ignite, CacheConfiguration metaCacheConfiguration,
+            CacheConfiguration dataCacheConfiguration, IDbCachingPolicy policy)
+        {
+            IgniteArgumentCheck.NotNull(ignite, "ignite");
+
+            metaCacheConfiguration = metaCacheConfiguration ?? GetDefaultMetaCacheConfiguration();
+            dataCacheConfiguration = dataCacheConfiguration ?? GetDefaultDataCacheConfiguration();
+
+            var efCache = new DbCache(ignite, metaCacheConfiguration, dataCacheConfiguration);
+
+            var txHandler = new DbTransactionInterceptor(efCache);
+
+            AddInterceptor(txHandler);
+
+            // SetProviderServices is not suitable. We should replace whatever provider there is with our proxy.
+            Loaded += (sender, args) => args.ReplaceService<DbProviderServices>(
+                (services, a) => new DbProviderServicesProxy(services, policy, efCache, txHandler));
+        }
+
+        /// <summary>
+        /// Gets the Ignite instance.
+        /// </summary>
+        private static IIgnite GetOrStartIgnite(IgniteConfiguration cfg)
+        {
+            cfg = cfg ?? new IgniteConfiguration();
+
+            return Ignition.TryGetIgnite(cfg.GridName) ?? Ignition.Start(cfg);
+        }
+
+        /// <summary>
+        /// Gets the configuration.
+        /// </summary>
+        private static IgniteConfiguration GetConfiguration(string sectionName, bool throwIfAbsent)
+        {
+            IgniteArgumentCheck.NotNull(sectionName, "sectionName");
+
+            var section = ConfigurationManager.GetSection(sectionName) as IgniteConfigurationSection;
+
+            if (section != null)
+            {
+                if (section.IgniteConfiguration == null)
+                    throw new IgniteException(string.Format(CultureInfo.InvariantCulture,
+                        "Failed to initialize {0}. {1} with name {2} is defined in <configSections>, " +
+                        "but not present in configuration.",
+                        typeof(IgniteDbConfiguration), typeof(IgniteConfigurationSection), sectionName));
+
+
+                return section.IgniteConfiguration;
+            }
+
+            if (!throwIfAbsent)
+                return null;
+
+            throw new IgniteException(string.Format(CultureInfo.InvariantCulture,
+                "Failed to initialize {0}. Could not find {1} with name {2} in application configuration.",
+                typeof (IgniteDbConfiguration), typeof (IgniteConfigurationSection), sectionName));
+        }
+
+        /// <summary>
+        /// Gets the default meta cache configuration.
+        /// </summary>
+        private static CacheConfiguration GetDefaultMetaCacheConfiguration(string namePrefix = null)
+        {
+            return new CacheConfiguration((namePrefix ?? DefaultCacheNamePrefix) + MetaCacheSuffix)
+            {
+                CacheMode = CacheMode.Partitioned,
+                Backups = 1,
+                AtomicityMode = CacheAtomicityMode.Transactional,  // Required due to IGNITE-3955
+                WriteSynchronizationMode = CacheWriteSynchronizationMode.PrimarySync
+            };
+        }
+
+        /// <summary>
+        /// Gets the default data cache configuration.
+        /// </summary>
+        private static CacheConfiguration GetDefaultDataCacheConfiguration(string namePrefix = null)
+        {
+            return new CacheConfiguration((namePrefix ?? DefaultCacheNamePrefix) + DataCacheSuffix)
+            {
+                CacheMode = CacheMode.Partitioned,
+                Backups = 0,
+                AtomicityMode = CacheAtomicityMode.Atomic,
+                WriteSynchronizationMode = CacheWriteSynchronizationMode.PrimarySync
+            };
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/ArrayDbDataReader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/ArrayDbDataReader.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/ArrayDbDataReader.cs
new file mode 100644
index 0000000..89523f4
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/ArrayDbDataReader.cs
@@ -0,0 +1,305 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.EntityFramework.Impl
+{
+    using System;
+    using System.Collections;
+    using System.Data;
+    using System.Data.Common;
+    using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
+
+    /// <summary>
+    /// Reads the data from array.
+    /// </summary>
+    internal class ArrayDbDataReader : DbDataReader
+    {
+        /** */
+        private readonly object[][] _data;
+
+        /** */
+        private readonly DataReaderField[] _schema;
+
+        /** */
+        private int _pos = -1;
+
+        /** */
+        private bool _closed;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ArrayDbDataReader"/> class.
+        /// </summary>
+        /// <param name="data">The data.</param>
+        /// <param name="schema">The schema.</param>
+        public ArrayDbDataReader(object[][] data, DataReaderField[] schema)
+        {
+            Debug.Assert(data != null);
+            Debug.Assert(schema != null);
+
+            _data = data;
+            _schema = schema;
+        }
+
+        /** <inheritDoc /> */
+        public override void Close()
+        {
+            _closed = true;
+        }
+
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        public override DataTable GetSchemaTable()
+        {
+            throw new NotSupportedException();
+        }
+
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        public override bool NextResult()
+        {
+            return false;  // multiple result sets are not supported
+        }
+
+        /** <inheritDoc /> */
+        public override bool Read()
+        {
+            if (_pos >= _data.Length - 1)
+                return false;
+
+            _pos++;
+
+            return true;
+        }
+
+        /** <inheritDoc /> */
+        public override int Depth
+        {
+            get { return 0; }
+        }
+
+        /** <inheritDoc /> */
+        public override bool IsClosed
+        {
+            get { return _closed; }
+        }
+
+        /** <inheritDoc /> */
+        public override int RecordsAffected
+        {
+            get { return -1; }
+        }
+
+        /** <inheritDoc /> */
+        public override bool GetBoolean(int ordinal)
+        {
+            return (bool) GetValue(ordinal);
+        }
+
+        /** <inheritDoc /> */
+        public override byte GetByte(int ordinal)
+        {
+            return (byte) GetValue(ordinal);
+        }
+
+        /** <inheritDoc /> */
+        public override long GetBytes(int ordinal, long dataOffset, byte[] buffer, int bufferOffset, int length)
+        {
+            Debug.Assert(buffer != null);
+
+            var data = (byte[]) GetValue(ordinal);
+
+            var size = Math.Min(buffer.Length - bufferOffset, data.Length - dataOffset);
+
+            Array.Copy(data, dataOffset, buffer, bufferOffset, size);
+
+            return size;
+        }
+
+        /** <inheritDoc /> */
+        public override char GetChar(int ordinal)
+        {
+            return (char) GetValue(ordinal);
+        }
+
+        /** <inheritDoc /> */
+        public override long GetChars(int ordinal, long dataOffset, char[] buffer, int bufferOffset, int length)
+        {
+            Debug.Assert(buffer != null);
+
+            var data = (char[]) GetValue(ordinal);
+
+            var size = Math.Min(buffer.Length - bufferOffset, data.Length - dataOffset);
+
+            Array.Copy(data, dataOffset, buffer, bufferOffset, size);
+
+            return size;
+        }
+
+        /** <inheritDoc /> */
+        public override Guid GetGuid(int ordinal)
+        {
+            return (Guid) GetValue(ordinal);
+        }
+
+        /** <inheritDoc /> */
+        public override short GetInt16(int ordinal)
+        {
+            return (short) GetValue(ordinal);
+        }
+
+        /** <inheritDoc /> */
+        public override int GetInt32(int ordinal)
+        {
+            return (int) GetValue(ordinal);
+        }
+
+        /** <inheritDoc /> */
+        public override long GetInt64(int ordinal)
+        {
+            return (long) GetValue(ordinal);
+        }
+
+        /** <inheritDoc /> */
+        public override DateTime GetDateTime(int ordinal)
+        {
+            return (DateTime) GetValue(ordinal);
+        }
+
+        /** <inheritDoc /> */
+        public override string GetString(int ordinal)
+        {
+            return (string) GetValue(ordinal);
+        }
+
+        /** <inheritDoc /> */
+        public override object GetValue(int ordinal)
+        {
+            return GetRow()[ordinal];
+        }
+
+        /** <inheritDoc /> */
+        public override int GetValues(object[] values)
+        {
+            var row = GetRow();
+
+            var size = Math.Min(row.Length, values.Length);
+
+            Array.Copy(row, values, size);
+
+            return size;
+        }
+
+        /** <inheritDoc /> */
+        public override bool IsDBNull(int ordinal)
+        {
+            var val = GetValue(ordinal);
+
+            return val == null || val == DBNull.Value;
+        }
+
+        /** <inheritDoc /> */
+        public override int FieldCount
+        {
+            get { return _schema.Length; }
+        }
+
+        /** <inheritDoc /> */
+        public override object this[int ordinal]
+        {
+            get { return GetValue(ordinal); }
+        }
+
+        /** <inheritDoc /> */
+        public override object this[string name]
+        {
+            get { return GetValue(GetOrdinal(name)); }
+        }
+
+        /** <inheritDoc /> */
+        public override bool HasRows
+        {
+            get { return _data.Length > 0; }
+        }
+
+        /** <inheritDoc /> */
+        public override decimal GetDecimal(int ordinal)
+        {
+            return (decimal) GetValue(ordinal);
+        }
+
+        /** <inheritDoc /> */
+        public override double GetDouble(int ordinal)
+        {
+            return (double) GetValue(ordinal);
+        }
+
+        /** <inheritDoc /> */
+        public override float GetFloat(int ordinal)
+        {
+            return (float) GetValue(ordinal);
+        }
+
+        /** <inheritDoc /> */
+        public override string GetName(int ordinal)
+        {
+            return _schema[ordinal].Name;
+        }
+
+        /** <inheritDoc /> */
+        public override int GetOrdinal(string name)
+        {
+            for (int i = 0; i < _schema.Length; i++)
+            {
+                if (_schema[i].Name == name)
+                    return i;
+            }
+
+            throw new InvalidOperationException("Field not found: " + name);
+        }
+
+        /** <inheritDoc /> */
+        public override string GetDataTypeName(int ordinal)
+        {
+            return _schema[ordinal].DataType;
+        }
+
+        /** <inheritDoc /> */
+        public override Type GetFieldType(int ordinal)
+        {
+            return _schema[ordinal].FieldType;
+        }
+
+        /** <inheritDoc /> */
+        [ExcludeFromCodeCoverage]
+        public override IEnumerator GetEnumerator()
+        {
+            throw new NotSupportedException();
+        }
+
+        /// <summary>
+        /// Gets the row.
+        /// </summary>
+        private object[] GetRow()
+        {
+            if (_pos < 0)
+                throw new InvalidOperationException("Data reading has not started.");
+
+            return _data[_pos];
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DataReaderField.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DataReaderField.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DataReaderField.cs
new file mode 100644
index 0000000..0e7baf0
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DataReaderField.cs
@@ -0,0 +1,74 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.EntityFramework.Impl
+{
+    using System;
+
+    /// <summary>
+    /// Represents a data reader field.
+    /// </summary>
+    [Serializable]
+    internal class DataReaderField
+    {
+        /** */
+        private readonly string _name;
+
+        /** */
+        private readonly Type _fieldType;
+
+        /** */
+        private readonly string _dataType;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="DataReaderField"/> class.
+        /// </summary>
+        /// <param name="name">The name.</param>
+        /// <param name="fieldType">The type.</param>
+        /// <param name="dataType">Type of the data.</param>
+        public DataReaderField(string name, Type fieldType, string dataType)
+        {
+            _name = name;
+            _fieldType = fieldType;
+            _dataType = dataType;
+        }
+
+        /// <summary>
+        /// Gets the name.
+        /// </summary>
+        public string Name
+        {
+            get { return _name; }
+        }
+
+        /// <summary>
+        /// Gets the type of the field.
+        /// </summary>
+        public Type FieldType
+        {
+            get { return _fieldType; }
+        }
+
+        /// <summary>
+        /// Gets the type of the data.
+        /// </summary>
+        public string DataType
+        {
+            get { return _dataType; }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5b31d83f/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DataReaderResult.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DataReaderResult.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DataReaderResult.cs
new file mode 100644
index 0000000..48f763c
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DataReaderResult.cs
@@ -0,0 +1,93 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.EntityFramework.Impl
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Data;
+    using System.Data.Common;
+    using System.Linq;
+
+    /// <summary>
+    /// Cacheable result of a DbDataReader.
+    /// </summary>
+    [Serializable]
+    internal class DataReaderResult
+    {
+        /** */
+        private readonly object[][] _data;
+
+        /** */
+        private readonly DataReaderField[] _schema;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="DataReaderResult"/> class.
+        /// </summary>
+        public DataReaderResult(IDataReader reader)
+        {
+            try
+            {
+                _data = ReadAll(reader).ToArray();
+
+                _schema = new DataReaderField[reader.FieldCount];
+
+                for (int i = 0; i < reader.FieldCount; i++)
+                {
+                    _schema[i] = new DataReaderField(reader.GetName(i), reader.GetFieldType(i), 
+                        reader.GetDataTypeName(i));
+                }
+            }
+            finally 
+            {
+                reader.Close();
+                reader.Dispose();
+            }
+        }
+
+        /// <summary>
+        /// Creates the reader over this instance.
+        /// </summary>
+        public DbDataReader CreateReader()
+        {
+            return new ArrayDbDataReader(_data, _schema);
+        }
+
+        /// <summary>
+        /// Gets the row count.
+        /// </summary>
+        public int RowCount
+        {
+            get { return _data.Length; }
+        }
+
+        /// <summary>
+        /// Reads all data from the reader.
+        /// </summary>
+        private static IEnumerable<object[]> ReadAll(IDataReader reader)
+        {
+            while (reader.Read())
+            {
+                var vals = new object[reader.FieldCount];
+
+                reader.GetValues(vals);
+
+                yield return vals;
+            }
+        }
+    }
+}


[43/50] [abbrv] ignite git commit: IGNITE-4186 .NET: Fix "Invalid session release request" exception in IgniteSessionStateStoreProvider.SetAndReleaseItemExclusive

Posted by vo...@apache.org.
IGNITE-4186 .NET: Fix "Invalid session release request" exception in IgniteSessionStateStoreProvider.SetAndReleaseItemExclusive

This closes #1227


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

Branch: refs/heads/ignite-2693
Commit: 884b281218d88c028daab25d35c14ee2b41be36e
Parents: 69487f2
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Thu Nov 10 20:35:15 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu Nov 10 20:38:07 2016 +0300

----------------------------------------------------------------------
 .../IgniteSessionStateStoreProviderTest.cs      | 49 +++++++++++++++++---
 .../IgniteSessionStateStoreProvider.cs          | 21 ++++++---
 2 files changed, 57 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/884b2812/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
index 9c3b07c..c6e3b30 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
@@ -209,6 +209,24 @@ namespace Apache.Ignite.AspNet.Tests
             Assert.AreEqual(TimeSpan.Zero, lockAge);
             Assert.AreEqual(SessionStateActions.None, actions);
 
+            // SetAndRelease with no lock. This happens with certain versions of ASP.NET.
+            var item = provider.CreateNewStoreData(HttpContext, 7);
+            // ReSharper disable once AssignNullToNotNullAttribute (lockId is not supposed to be null, but it can be).
+            provider.SetAndReleaseItemExclusive(HttpContext, Id, item, null, true);
+
+            // Check added item.
+            res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions);
+            Assert.IsNotNull(res);
+            Assert.IsNull(lockId);
+            Assert.AreEqual(7, res.Timeout);
+            Assert.IsFalse(locked);
+            Assert.AreEqual(TimeSpan.Zero, lockAge);
+            Assert.AreEqual(SessionStateActions.None, actions);
+
+            // Remove item.
+            // ReSharper disable once AssignNullToNotNullAttribute (lockId is not supposed to be null, but it can be).
+            provider.RemoveItem(HttpContext, Id, null, null);
+
             // Add item.
             provider.CreateUninitializedItem(HttpContext, Id, 7);
             
@@ -228,7 +246,7 @@ namespace Apache.Ignite.AspNet.Tests
             Assert.IsFalse(locked);
             Assert.AreEqual(TimeSpan.Zero, lockAge);
             Assert.AreEqual(SessionStateActions.None, actions);
-            provider.SetAndReleaseItemExclusive(HttpContext, Id, UpdateStoreData(res), lockId, true);
+            provider.SetAndReleaseItemExclusive(HttpContext, Id, UpdateStoreData(res), lockId, false);
 
             // Not locked, item present.
             res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions);
@@ -331,20 +349,37 @@ namespace Apache.Ignite.AspNet.Tests
             Assert.IsFalse(GetProvider().SetItemExpireCallback(null));
 
             // Check there is no item.
-            var res = provider.GetItem(HttpContext, "myId", out locked, out lockAge, out lockId, out actions);
+            var res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions);
             Assert.IsNull(res);
 
-            // Put an item.
-            provider.CreateUninitializedItem(HttpContext, "myId", 1);
+            // Put an item with CreateUninitializedItem and check.
+            provider.CreateUninitializedItem(HttpContext, Id, 1);
+            CheckExpiry(provider);
+
+            // Put an item with SetAndReleaseItemExclusive and check.
+            var data = provider.CreateNewStoreData(HttpContext, 1);
+            provider.SetAndReleaseItemExclusive(HttpContext, Id, data, lockId, true);
+            CheckExpiry(provider);
+        }
 
-            // Check that it is there.
-            res = provider.GetItem(HttpContext, "myId", out locked, out lockAge, out lockId, out actions);
+        /// <summary>
+        /// Checks item expiration.
+        /// </summary>
+        private static void CheckExpiry(SessionStateStoreProviderBase provider)
+        {
+            bool locked;
+            TimeSpan lockAge;
+            object lockId;
+            SessionStateActions actions;
+
+            // Check that item is present.
+            var res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions);
             Assert.IsNotNull(res);
 
             // Wait a minute and check again.
             Thread.Sleep(TimeSpan.FromMinutes(1.05));
 
-            res = provider.GetItem(HttpContext, "myId", out locked, out lockAge, out lockId, out actions);
+            res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions);
             Assert.IsNull(res);
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/884b2812/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs
index 86035dd..a5e8199 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/IgniteSessionStateStoreProvider.cs
@@ -294,11 +294,20 @@ namespace Apache.Ignite.AspNet
 
             var data = (IgniteSessionStateStoreData) item;
 
-            if (!(lockId is long) || data.LockId != (long) lockId)
-                throw new IgniteException(string.Format(CultureInfo.InvariantCulture,
-                    "Invalid session release request, expected lockId: {0}, actual: {1}", data.LockId, lockId));
+            if (newItem)
+            {
+                var cache = _expiryCacheHolder.GetCacheWithExpiry(data.Timeout * 60);
+
+                PutItem(key, data, cache);
+            }
+            else
+            {
+                if (!(lockId is long) || data.LockId != (long) lockId)
+                    throw new IgniteException(string.Format(CultureInfo.InvariantCulture,
+                        "Invalid session release request, expected lockId: {0}, actual: {1}", data.LockId, lockId));
 
-            SetAndUnlockItem(key, data);
+                SetAndUnlockItem(key, data);
+            }
         }
 
         /// <summary>
@@ -333,7 +342,7 @@ namespace Apache.Ignite.AspNet
         /// </summary>
         /// <param name="context">The <see cref="T:System.Web.HttpContext" /> for the current request.</param>
         /// <param name="timeout">The session-state <see cref="P:System.Web.SessionState.HttpSessionState.Timeout" /> 
-        /// value for the new <see cref="T:System.Web.SessionState.SessionStateStoreData" />.</param>
+        /// value for the new <see cref="T:System.Web.SessionState.SessionStateStoreData" />, in minutes.</param>
         /// <returns>
         /// A new <see cref="T:System.Web.SessionState.SessionStateStoreData" /> for the current request.
         /// </returns>
@@ -349,7 +358,7 @@ namespace Apache.Ignite.AspNet
         /// <param name="id">The <see cref="P:System.Web.SessionState.HttpSessionState.SessionID" /> 
         /// for the current request.</param>
         /// <param name="timeout">The session <see cref="P:System.Web.SessionState.HttpSessionState.Timeout" /> 
-        /// for the current request.</param>
+        /// for the current request, in minutes.</param>
         public override void CreateUninitializedItem(HttpContext context, string id, int timeout)
         {
             var cache = _expiryCacheHolder.GetCacheWithExpiry((long) timeout * 60);


[02/50] [abbrv] ignite git commit: Fixes after merge.

Posted by vo...@apache.org.
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/b8b9abe8
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/b8b9abe8
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/b8b9abe8

Branch: refs/heads/ignite-2693
Commit: b8b9abe863ed8139553a9ad7013dfad5a363b4da
Parents: 5fac786
Author: devozerov <vo...@gridgain.com>
Authored: Mon Oct 31 21:31:22 2016 +0300
Committer: thatcoach <pp...@list.ru>
Committed: Mon Oct 31 21:59:09 2016 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/cache/query/SqlQuery.java |   5 +-
 .../processors/cache/QueryCursorImpl.java       |  18 +-
 .../closure/GridClosureProcessor.java           |   1 -
 .../processors/query/GridQueryCancel.java       |  60 ++--
 .../twostep/messages/GridQueryFailResponse.java |  13 +-
 .../junits/GridTestKernalContext.java           |  14 +-
 .../query/h2/twostep/GridMergeIndex.java        |  12 +-
 .../h2/twostep/msg/GridH2QueryRequest.java      |  42 ++-
 ...butedQueryStopOnCancelOrTimeoutSelfTest.java |  57 ++--
 ...cheQueryAbstractDistributedJoinSelfTest.java | 290 ++++++++++++++++++-
 ...QueryNodeRestartDistributedJoinSelfTest.java | 262 +----------------
 .../IgniteCacheQueryNodeRestartSelfTest2.java   |   2 +-
 ...nCancelOrTimeoutDistributedJoinSelfTest.java | 137 ++++++++-
 .../IgniteCacheQuerySelfTestSuite.java          |   4 +-
 .../IgniteCacheQuerySelfTestSuite2.java         |   2 +
 15 files changed, 564 insertions(+), 355 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b8b9abe8/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 83e171d..3b8fe6d 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
@@ -17,14 +17,15 @@
 
 package org.apache.ignite.cache.query;
 
-import java.util.concurrent.TimeUnit;
-import javax.cache.Cache;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
+import javax.cache.Cache;
+import java.util.concurrent.TimeUnit;
+
 /**
  * SQL Query.
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8b9abe8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
index f68426e..f93a747 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
@@ -17,11 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
-import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.query.QueryCancelledException;
@@ -29,10 +24,13 @@ import org.apache.ignite.internal.processors.cache.query.QueryCursorEx;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 
-import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.CLOSED;
-import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.EXECUTION;
-import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.IDLE;
-import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.RESULT_READY;
+import javax.cache.CacheException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+
+import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.*;
 
 /**
  * Query cursor implementation.
@@ -40,7 +38,7 @@ import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.
 public class QueryCursorImpl<T> implements QueryCursorEx<T> {
     /** */
     private final static AtomicReferenceFieldUpdater<QueryCursorImpl, State> STATE_UPDATER =
-            AtomicReferenceFieldUpdater.newUpdater(QueryCursorImpl.class, State.class, "state");
+        AtomicReferenceFieldUpdater.newUpdater(QueryCursorImpl.class, State.class, "state");
 
     /** Query executor. */
     private Iterable<T> iterExec;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8b9abe8/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 252540e..9d295d3 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
@@ -27,7 +27,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
-import java.util.concurrent.Executor;
 import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.TimeUnit;
 import org.apache.ignite.IgniteCheckedException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8b9abe8/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java
index 47f1208..7391f39 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java
@@ -18,49 +18,57 @@
 package org.apache.ignite.internal.processors.query;
 
 import org.apache.ignite.cache.query.QueryCancelledException;
-import org.apache.ignite.internal.IgniteInterruptedCheckedException;
-import org.apache.ignite.internal.util.typedef.internal.U;
+
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
 /**
  * Holds query cancel state.
  */
 public class GridQueryCancel {
-    /** */
-    private volatile boolean cancelled;
+    /** No-op runnable indicating cancelled state. */
+    private static final Runnable CANCELLED = new Runnable() {
+        @Override public void run() {
+            // No-op.
+        }
+    };
 
     /** */
-    private volatile boolean completed;
+    private static final AtomicReferenceFieldUpdater<GridQueryCancel, Runnable> STATE_UPDATER =
+        AtomicReferenceFieldUpdater.newUpdater(GridQueryCancel.class, Runnable.class, "clo");
 
     /** */
     private volatile Runnable clo;
 
     /**
-     * Sets a cancel closure. The closure must be idempotent to multiple invocations.
+     * Sets a cancel closure.
      *
      * @param clo Clo.
      */
-    public void set(Runnable clo) throws QueryCancelledException{
-        checkCancelled();
+    public void set(Runnable clo) throws QueryCancelledException {
+        assert clo != null;
 
-        this.clo = clo;
+        while(true) {
+            Runnable tmp = this.clo;
+
+            if (tmp == CANCELLED)
+                throw new QueryCancelledException();
+
+            if (STATE_UPDATER.compareAndSet(this, tmp, clo))
+                return;
+        }
     }
 
     /**
-     * Spins until a query is completed.
-     * Only one thread can enter this method.
-     * This is guaranteed by {@link org.apache.ignite.internal.processors.cache.QueryCursorImpl}
+     * Executes cancel closure.
      */
     public void cancel() {
-        cancelled = true;
-
-        int attempt = 0;
+        while(true) {
+            Runnable tmp = this.clo;
 
-        while (!completed) {
-            if (clo != null) clo.run();
+            if (STATE_UPDATER.compareAndSet(this, tmp, CANCELLED)) {
+                if (tmp != null)
+                    tmp.run();
 
-            try {
-                U.sleep(++attempt * 10);
-            } catch (IgniteInterruptedCheckedException ignored) {
                 return;
             }
         }
@@ -69,16 +77,8 @@ public class GridQueryCancel {
     /**
      * Stops query execution if a user requested cancel.
      */
-    public void checkCancelled() throws QueryCancelledException{
-        if (cancelled)
+    public void checkCancelled() throws QueryCancelledException {
+        if (clo == CANCELLED)
             throw new QueryCancelledException();
     }
-
-    /**
-     * Sets completed state.
-     * The method must be called then a query is completed by any reason, typically in final block.
-     */
-    public void setCompleted() {
-        completed = true;
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8b9abe8/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 261241e..7554ae9 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
@@ -17,13 +17,14 @@
 
 package org.apache.ignite.internal.processors.query.h2.twostep.messages;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.cache.query.QueryCancelledException;
 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;
 
+import java.nio.ByteBuffer;
+
 /**
  * Error message.
  */
@@ -113,13 +114,13 @@ public class GridQueryFailResponse implements Message {
                 writer.incrementState();
 
             case 1:
-                if (!writer.writeByte("failCode", failCode))
+                if (!writer.writeLong("qryReqId", qryReqId))
                     return false;
 
                 writer.incrementState();
 
             case 2:
-                if (!writer.writeLong("qryReqId", qryReqId))
+                if (!writer.writeByte("failCode", failCode))
                     return false;
 
                 writer.incrementState();
@@ -146,7 +147,7 @@ public class GridQueryFailResponse implements Message {
                 reader.incrementState();
 
             case 1:
-                failCode = reader.readByte("failCode");
+                qryReqId = reader.readLong("qryReqId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -154,7 +155,7 @@ public class GridQueryFailResponse implements Message {
                 reader.incrementState();
 
             case 2:
-                qryReqId = reader.readLong("qryReqId");
+                failCode = reader.readByte("failCode");
 
                 if (!reader.isLastRead())
                     return false;
@@ -175,4 +176,4 @@ public class GridQueryFailResponse implements Message {
     @Override public byte fieldsCount() {
         return 3;
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8b9abe8/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
index cba67e0..03138c3 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
@@ -17,21 +17,18 @@
 
 package org.apache.ignite.testframework.junits;
 
-import java.util.List;
-import java.util.ListIterator;
-import java.util.concurrent.ExecutorService;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.GridComponent;
-import org.apache.ignite.internal.GridKernalContextImpl;
-import org.apache.ignite.internal.GridKernalGatewayImpl;
-import org.apache.ignite.internal.GridLoggerProxy;
-import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 
+import java.util.List;
+import java.util.ListIterator;
+import java.util.concurrent.ExecutorService;
+
 /**
  * Test context.
  */
@@ -62,6 +59,7 @@ public class GridTestKernalContext extends GridKernalContextImpl {
                 null,
                 null,
                 null,
+                null,
                 U.allPluginProviders());
 
         GridTestUtils.setFieldValue(grid(), "cfg", config());

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8b9abe8/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 796ea66..444ea82 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
@@ -20,7 +20,6 @@ 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;
@@ -113,7 +112,7 @@ public abstract class GridMergeIndex extends BaseIndex {
     protected final void checkSourceNodesAlive() {
         for (UUID nodeId : sources()) {
             if (!ctx.discovery().alive(nodeId)) {
-                fail(nodeId);
+                fail(nodeId, null);
 
                 return;
             }
@@ -174,11 +173,18 @@ public abstract class GridMergeIndex extends BaseIndex {
     /**
      * @param nodeId Node ID.
      */
-    public void fail(UUID nodeId) {
+    public void fail(UUID nodeId, final CacheException e) {
         addPage0(new GridResultPage(null, nodeId, null) {
             @Override public boolean isFail() {
                 return true;
             }
+
+            @Override public void fetchNextPage() {
+                if (e == null)
+                    super.fetchNextPage();
+                else
+                    throw e;
+            }
         });
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8b9abe8/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 dc82b2c..884173f 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
@@ -45,8 +45,8 @@ public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable {
     private static final long serialVersionUID = 0L;
 
     /**
-     * Map query will not destroy context until explicit query cancel request
-     * will be received because distributed join requests can be received.
+     * Map query will not destroy context until explicit query cancel request will be received because distributed join
+     * requests can be received.
      */
     public static int FLAG_DISTRIBUTED_JOINS = 1;
 
@@ -82,6 +82,9 @@ public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable {
     @GridDirectCollection(String.class)
     private Collection<String> tbls;
 
+    /** */
+    private int timeout;
+
     /**
      * @param tbls Tables.
      * @return {@code this}.
@@ -153,7 +156,7 @@ public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable {
     /**
      * @return Explicit partitions mapping.
      */
-    public Map<UUID,int[]> partitions() {
+    public Map<UUID, int[]> partitions() {
         return parts;
     }
 
@@ -161,7 +164,7 @@ public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable {
      * @param parts Explicit partitions mapping.
      * @return {@code this}.
      */
-    public GridH2QueryRequest partitions(Map<UUID,int[]> parts) {
+    public GridH2QueryRequest partitions(Map<UUID, int[]> parts) {
         this.parts = parts;
 
         return this;
@@ -219,6 +222,23 @@ public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable {
         return (this.flags & flags) == flags;
     }
 
+    /**
+     * @return Timeout.
+     */
+    public int timeout() {
+        return timeout;
+    }
+
+    /**
+     * @param timeout New timeout.
+     * @return {@code this}.
+     */
+    public GridH2QueryRequest timeout(int timeout) {
+        this.timeout = timeout;
+
+        return this;
+    }
+
     /** {@inheritDoc} */
     @Override public void marshall(Marshaller m) {
         if (F.isEmpty(qrys))
@@ -297,6 +317,11 @@ public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable {
 
                 writer.incrementState();
 
+            case 8:
+                if (!writer.writeInt("timeout", timeout))
+                    return false;
+
+                writer.incrementState();
         }
 
         return true;
@@ -374,6 +399,13 @@ public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable {
 
                 reader.incrementState();
 
+            case 8:
+                timeout = reader.readInt("timeout");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
         }
 
         return reader.afterMessageRead(GridH2QueryRequest.class);
@@ -386,7 +418,7 @@ public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 8;
+        return 9;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8b9abe8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java
index 0f60db2..a92bf2b 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import java.util.Arrays;
 import java.util.List;
+import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
@@ -43,7 +44,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
  */
 public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends GridCommonAbstractTest {
     /** Grids count. */
-    private static final int GRIDS_COUNT = 3;
+    private static final int GRIDS_CNT = 3;
 
     /** IP finder. */
     private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
@@ -55,19 +56,19 @@ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends Gr
     public static final int VAL_SIZE = 16;
 
     /** */
-    private static final String QUERY_1 = "select a._val, b._val from String a, String b";
+    private static final String QRY_1 = "select a._val, b._val from String a, String b";
 
     /** */
-    private static final String QUERY_2 = "select a._key, count(*) from String a group by a._key";
+    private static final String QRY_2 = "select a._key, count(*) from String a group by a._key";
 
     /** */
-    private static final String QUERY_3 = "select a._val from String a";
+    private static final String QRY_3 = "select a._val from String a";
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();
 
-        startGridsMultiThreaded(GRIDS_COUNT);
+        startGridsMultiThreaded(GRIDS_CNT);
     }
 
     /** {@inheritDoc} */
@@ -97,82 +98,82 @@ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends Gr
 
     /** */
     public void testRemoteQueryExecutionTimeout() throws Exception {
-        testQuery(CACHE_SIZE, VAL_SIZE, QUERY_1, 500, TimeUnit.MILLISECONDS, true);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 500, TimeUnit.MILLISECONDS, true);
     }
 
     /** */
     public void testRemoteQueryWithMergeTableTimeout() throws Exception {
-        testQuery(CACHE_SIZE, VAL_SIZE, QUERY_2, 500, TimeUnit.MILLISECONDS, true);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 500, TimeUnit.MILLISECONDS, true);
     }
 
     /** */
     public void testRemoteQueryExecutionCancel0() throws Exception {
-        testQuery(CACHE_SIZE, VAL_SIZE, QUERY_1, 1, TimeUnit.MILLISECONDS, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testRemoteQueryExecutionCancel1() throws Exception {
-        testQuery(CACHE_SIZE, VAL_SIZE, QUERY_1, 500, TimeUnit.MILLISECONDS, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 500, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testRemoteQueryExecutionCancel2() throws Exception {
-        testQuery(CACHE_SIZE, VAL_SIZE, QUERY_1, 1, TimeUnit.SECONDS, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 1, TimeUnit.SECONDS, false);
     }
 
     /** */
     public void testRemoteQueryExecutionCancel3() throws Exception {
-        testQuery(CACHE_SIZE, VAL_SIZE, QUERY_1, 3, TimeUnit.SECONDS, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_1, 3, TimeUnit.SECONDS, false);
     }
 
     /** */
     public void testRemoteQueryWithMergeTableCancel0() throws Exception {
-        testQuery(CACHE_SIZE, VAL_SIZE, QUERY_2, 1, TimeUnit.MILLISECONDS, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testRemoteQueryWithMergeTableCancel1() throws Exception {
-        testQuery(CACHE_SIZE, VAL_SIZE, QUERY_2, 500, TimeUnit.MILLISECONDS, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 500, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testRemoteQueryWithMergeTableCancel2() throws Exception {
-        testQuery(CACHE_SIZE, VAL_SIZE, QUERY_2, 1_500, TimeUnit.MILLISECONDS, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 1_500, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testRemoteQueryWithMergeTableCancel3() throws Exception {
-        testQuery(CACHE_SIZE, VAL_SIZE, QUERY_2, 3, TimeUnit.SECONDS, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_2, 3, TimeUnit.SECONDS, false);
     }
 
     /** */
     public void testRemoteQueryWithoutMergeTableCancel0() throws Exception {
-        testQuery(CACHE_SIZE, VAL_SIZE, QUERY_3, 1, TimeUnit.MILLISECONDS, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 1, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testRemoteQueryWithoutMergeTableCancel1() throws Exception {
-        testQuery(CACHE_SIZE, VAL_SIZE, QUERY_3, 500, TimeUnit.MILLISECONDS, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 500, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testRemoteQueryWithoutMergeTableCancel2() throws Exception {
-        testQuery(CACHE_SIZE, VAL_SIZE, QUERY_3, 1_000, TimeUnit.MILLISECONDS, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 1_000, TimeUnit.MILLISECONDS, false);
     }
 
     /** */
     public void testRemoteQueryWithoutMergeTableCancel3() throws Exception {
-        testQuery(CACHE_SIZE, VAL_SIZE, QUERY_3, 3, TimeUnit.SECONDS, false);
+        testQueryCancel(CACHE_SIZE, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false);
     }
 
     /** */
     public void testRemoteQueryAlreadyFinishedStop() throws Exception {
-        testQuery(100, VAL_SIZE, QUERY_3, 3, TimeUnit.SECONDS, false);
+        testQueryCancel(100, VAL_SIZE, QRY_3, 3, TimeUnit.SECONDS, false);
     }
 
     /** */
-    private void testQuery(int keyCnt, int valSize, String sql, int timeoutUnits, TimeUnit timeUnit,
-        boolean timeout) throws Exception {
+    private void testQueryCancel(int keyCnt, int valSize, String sql, int timeoutUnits, TimeUnit timeUnit,
+                                 boolean timeout) throws Exception {
         try (Ignite client = startGrid("client")) {
 
             IgniteCache<Object, Object> cache = client.cache(null);
@@ -230,19 +231,23 @@ public class IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest extends Gr
     /**
      * Validates clean state on all participating nodes after query cancellation.
      */
+    @SuppressWarnings("unchecked")
     private void checkCleanState() {
-        for (int i = 0; i < GRIDS_COUNT; i++) {
+        for (int i = 0; i < GRIDS_CNT; i++) {
             IgniteEx grid = grid(i);
 
             // Validate everything was cleaned up.
-            ConcurrentMap<UUID, ConcurrentMap<Long, ?>> map = U.field(((IgniteH2Indexing)U.field((Object)U.field(
+            ConcurrentMap<UUID, ?> map = U.field(((IgniteH2Indexing)U.field(U.field(
                 grid.context(), "qryProc"), "idx")).mapQueryExecutor(), "qryRess");
 
             String msg = "Map executor state is not cleared";
 
             // TODO FIXME Current implementation leaves map entry for each node that's ever executed a query.
-            for (ConcurrentMap<Long, ?> results : map.values())
-                assertEquals(msg, 0, results.size());
+            for (Object result : map.values()) {
+                Map<Long, ?> m = U.field(result, "res");
+
+                assertEquals(msg, 0, m.size());
+            }
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8b9abe8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java
index be34a09..339e0d3 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java
@@ -1,7 +1,291 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.cache.query.annotations.QuerySqlFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.util.GridRandom;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import java.io.Serializable;
+import java.util.Random;
+
+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;
+
 /**
- * Created by vozerov on 31.10.2016.
+ * Test for distributed queries with node restarts.
  */
-public class IgniteCacheQueryAbstractDistributedJoinSelfTest {
-}
+public class IgniteCacheQueryAbstractDistributedJoinSelfTest extends GridCommonAbstractTest {
+    /** */
+    protected 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._key = pu.personId and pu.productId = pr._key and pr.companyId = co._key \n" +
+        "group by co._key order by cnt desc, co._key";
+
+    /** */
+    protected static final String QRY_0_BROADCAST = "select co._key, count(*) cnt\n" +
+        "from \"co\".Company co, \"pr\".Product pr, \"pu\".Purchase pu, \"pe\".Person pe \n" +
+        "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";
+
+    /** */
+    protected static final String QRY_1 = "select pr._key, co._key\n" +
+        "from \"pr\".Product pr, \"co\".Company co\n" +
+        "where pr.companyId = co._key\n" +
+        "order by co._key, pr._key ";
+
+    /** */
+    protected static final String QRY_1_BROADCAST = "select pr._key, co._key\n" +
+        "from \"co\".Company co, \"pr\".Product pr \n" +
+        "where pr.companyId = co._key\n" +
+        "order by co._key, pr._key ";
+
+    /** */
+    protected static final int GRID_CNT = 2;
+
+    /** */
+    private static final int PERS_CNT = 600;
+
+    /** */
+    private static final int PURCHASE_CNT = 6_000;
+
+    /** */
+    private static final int COMPANY_CNT = 25;
+
+    /** */
+    private static final int PRODUCT_CNT = 100;
+
+    /** */
+    private static TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(gridName);
+
+        if ("client".equals(gridName))
+            c.setClientMode(true);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        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;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridsMultiThreaded(GRID_CNT);
+
+        fillCaches();
+    }
+
+    /**
+     *
+     */
+    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));
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     *
+     */
+    protected static class Person implements Serializable {
+        /** */
+        @QuerySqlField(index = true)
+        int id;
+
+        /**
+         * @param id ID.
+         */
+        Person(int id) {
+            this.id = id;
+        }
+    }
+
+    /**
+     *
+     */
+    protected static class Purchase implements Serializable {
+        /** */
+        @QuerySqlField(index = true)
+        int personId;
+
+        /** */
+        @QuerySqlField(index = true)
+        int productId;
+
+        /**
+         * @param personId Person ID.
+         * @param productId Product ID.
+         */
+        Purchase(int personId, int productId) {
+            this.personId = personId;
+            this.productId = productId;
+        }
+    }
+
+    /**
+     *
+     */
+    protected static class Company implements Serializable {
+        /** */
+        @QuerySqlField(index = true)
+        int id;
+
+        /**
+         * @param id ID.
+         */
+        Company(int id) {
+            this.id = id;
+        }
+    }
+
+    /**
+     *
+     */
+    protected static class Product implements Serializable {
+        /** */
+        @QuerySqlField(index = true)
+        int id;
+
+        /** */
+        @QuerySqlField(index = true)
+        int companyId;
+
+        /**
+         * @param id ID.
+         * @param companyId Company ID.
+         */
+        Product(int id, int companyId) {
+            this.id = id;
+            this.companyId = companyId;
+        }
+    }
+
+    /** */
+    public static class Functions {
+        /** */
+        @QuerySqlFunction
+        public static int sleep() {
+            try {
+                U.sleep(1_000);
+            } catch (IgniteInterruptedCheckedException ignored) {
+                // No-op.
+            }
+
+            return 0;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8b9abe8/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 0e6806f..ced28bc 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
@@ -17,185 +17,25 @@
 
 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.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;
-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;
+import javax.cache.CacheException;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicIntegerArray;
 
 /**
  * Test for distributed queries with node restarts.
  */
-public class IgniteCacheQueryNodeRestartDistributedJoinSelfTest extends GridCommonAbstractTest {
-    /** */
-    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._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_0_BROADCAST = "select co._key, count(*) cnt\n" +
-        "from \"co\".Company co, \"pr\".Product pr, \"pu\".Purchase pu, \"pe\".Person pe \n" +
-        "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._key, co._key\n" +
-        "from \"pr\".Product pr, \"co\".Company co\n" +
-        "where pr.companyId = co._key\n" +
-        "order by co._key, pr._key ";
-
-    /** */
-    private static final String QRY_1_BROADCAST = "select pr._key, co._key\n" +
-        "from \"co\".Company co, \"pr\".Product pr \n" +
-        "where pr.companyId = co._key\n" +
-        "order by co._key, pr._key ";
-
-    /** */
-    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;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
-        startGridsMultiThreaded(GRID_CNT);
-
-        fillCaches();
-    }
-
-    /**
-     *
-     */
-    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));
-        }
-    }
+public class IgniteCacheQueryNodeRestartDistributedJoinSelfTest extends IgniteCacheQueryAbstractDistributedJoinSelfTest {
     /**
      * @throws Exception If failed.
      */
@@ -319,13 +159,6 @@ public class IgniteCacheQueryNodeRestartDistributedJoinSelfTest extends GridComm
                     else {
                         IgniteCache<?, ?> cache = grid(g).cache("co");
 
-                        SqlFieldsQuery qry;
-
-                        if (broadcastQry)
-                            qry = new SqlFieldsQuery(QRY_1_BROADCAST).setDistributedJoins(true).setEnforceJoinOrder(true);
-                        else
-                            qry = new SqlFieldsQuery(QRY_1).setDistributedJoins(true);
-
                         assertEquals(rRes, cache.query(qry1).getAll());
                     }
 
@@ -392,85 +225,4 @@ public class IgniteCacheQueryNodeRestartDistributedJoinSelfTest extends GridComm
 
         info("Stopped.");
     }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /**
-     *
-     */
-    private static class Person implements Serializable {
-        /** */
-        @QuerySqlField(index = true)
-        int id;
-
-        /**
-         * @param id ID.
-         */
-        Person(int id) {
-            this.id = id;
-        }
-    }
-
-    /**
-     *
-     */
-    private static class Purchase implements Serializable {
-        /** */
-        @QuerySqlField(index = true)
-        int personId;
-
-        /** */
-        @QuerySqlField(index = true)
-        int productId;
-
-        /**
-         * @param personId Person ID.
-         * @param productId Product ID.
-         */
-        Purchase(int personId, int productId) {
-            this.personId = personId;
-            this.productId = productId;
-        }
-    }
-
-    /**
-     *
-     */
-    private static class Company implements Serializable {
-        /** */
-        @QuerySqlField(index = true)
-        int id;
-
-        /**
-         * @param id ID.
-         */
-        Company(int id) {
-            this.id = id;
-        }
-    }
-
-    /**
-     *
-     */
-    private static class Product implements Serializable {
-        /** */
-        @QuerySqlField(index = true)
-        int id;
-
-        /** */
-        @QuerySqlField(index = true)
-        int companyId;
-
-        /**
-         * @param id ID.
-         * @param companyId Company ID.
-         */
-        Product(int id, int companyId) {
-            this.id = id;
-            this.companyId = companyId;
-        }
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8b9abe8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
index 8b33a46..154daa0 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
@@ -267,7 +267,7 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest
                                         continue;
 
                                     if (th.getMessage() != null &&
-                                            th.getMessage().startsWith("Failed to fetch data from node:")) {
+                                        th.getMessage().startsWith("Failed to fetch data from node:")) {
                                         failedOnRemoteFetch = true;
 
                                         break;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8b9abe8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java
index 80bd62e..4baaf8f 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.java
@@ -1,7 +1,138 @@
+/*
+ * 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.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import javax.cache.CacheException;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.QueryCancelledException;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
 /**
- * Created by vozerov on 31.10.2016.
+ * Test for cancel of query containing distributed joins.
  */
-public class IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest {
-}
+public class IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest extends IgniteCacheQueryAbstractDistributedJoinSelfTest {
+    /** */
+    public void testCancel1() throws Exception {
+        testQueryCancel(grid(0), "pe", QRY_0, 1, TimeUnit.MILLISECONDS, false);
+    }
+
+    /** */
+    public void testCancel2() throws Exception {
+        testQueryCancel(grid(0), "pe", QRY_0, 50, TimeUnit.MILLISECONDS, false);
+    }
+
+    /** */
+    public void testCancel3() throws Exception {
+        testQueryCancel(grid(0), "pe", QRY_0, 100, TimeUnit.MILLISECONDS, false);
+    }
+
+    /** */
+    public void testCancel4() throws Exception {
+        testQueryCancel(grid(0), "pe", QRY_0, 500, TimeUnit.MILLISECONDS, false);
+    }
+
+    /** */
+    public void testTimeout1() throws Exception {
+        testQueryCancel(grid(0), "pe", QRY_0, 1, TimeUnit.MILLISECONDS, true);
+    }
+
+    /** */
+    public void testTimeout2() throws Exception {
+        testQueryCancel(grid(0), "pe", QRY_0, 50, TimeUnit.MILLISECONDS, true);
+    }
+
+    /** */
+    public void testTimeout3() throws Exception {
+        testQueryCancel(grid(0), "pe", QRY_0, 100, TimeUnit.MILLISECONDS, true);
+    }
+
+    /** */
+    public void testTimeout4() throws Exception {
+        testQueryCancel(grid(0), "pe", QRY_0, 500, TimeUnit.MILLISECONDS, true);
+    }
+
+    /** */
+    private void testQueryCancel(Ignite ignite, String cacheName, String sql, int timeoutUnits, TimeUnit timeUnit,
+                           boolean timeout) throws Exception {
+        SqlFieldsQuery qry = new SqlFieldsQuery(sql).setDistributedJoins(true);
+
+        IgniteCache<Object, Object> cache = ignite.cache(cacheName);
+
+        final QueryCursor<List<?>> cursor;
+        if (timeout) {
+            qry.setTimeout(timeoutUnits, timeUnit);
+
+            cursor = cache.query(qry);
+        } else {
+            cursor = cache.query(qry);
+
+            ignite.scheduler().runLocal(new Runnable() {
+                @Override public void run() {
+                    cursor.close();
+                }
+            }, timeoutUnits, timeUnit);
+        }
+
+        try (QueryCursor<List<?>> ignored = cursor) {
+            cursor.iterator();
+        }
+        catch (CacheException ex) {
+            log().error("Got expected exception", ex);
+
+            assertTrue("Must throw correct exception", ex.getCause() instanceof QueryCancelledException);
+        }
+
+        // Give some time to clean up.
+        Thread.sleep(TimeUnit.MILLISECONDS.convert(timeoutUnits, timeUnit) + 3_000);
+
+        checkCleanState();
+    }
+
+    /**
+     * Validates clean state on all participating nodes after query cancellation.
+     */
+    @SuppressWarnings("unchecked")
+    private void checkCleanState() {
+        for (int i = 0; i < GRID_CNT; i++) {
+            IgniteEx grid = grid(i);
+
+            // Validate everything was cleaned up.
+            ConcurrentMap<UUID, ?> map = U.field(((IgniteH2Indexing) U.field(U.field(
+                    grid.context(), "qryProc"), "idx")).mapQueryExecutor(), "qryRess");
+
+            String msg = "Map executor state is not cleared";
+
+            // TODO FIXME Current implementation leaves map entry for each node that's ever executed a query.
+            for (Object result : map.values()) {
+                Map<Long, ?> m = U.field(result, "res");
+
+                assertEquals(msg, 0, m.size());
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8b9abe8/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 b7e6403..7f98d0a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -63,7 +63,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheP
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedSnapshotEnabledQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNoRebalanceSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeFailTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartDistributedJoinSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryAbstractDistributedJoinSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest2;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQueryP2PDisabledSelfTest;
@@ -125,7 +125,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheOffheapIndexScanTest.class);
         suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest2.class);
-        suite.addTestSuite(IgniteCacheQueryNodeRestartDistributedJoinSelfTest.class);
+        suite.addTestSuite(IgniteCacheQueryAbstractDistributedJoinSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryNodeFailTest.class);
         suite.addTestSuite(IgniteCacheClientQueryReplicatedNodeRestartSelfTest.class);
         suite.addTestSuite(GridCacheCrossCacheQuerySelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8b9abe8/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
index 5722c01..be7523f 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
@@ -38,6 +38,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheD
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQueryP2PEnabledSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest;
@@ -100,6 +101,7 @@ public class IgniteCacheQuerySelfTestSuite2 extends TestSuite {
         suite.addTestSuite(IgniteCacheDistributedQueryStopOnCancelOrTimeoutSelfTest.class);
         suite.addTestSuite(IgniteCacheDistributedQueryCancelSelfTest.class);
         suite.addTestSuite(IgniteCacheLocalQueryCancelOrTimeoutSelfTest.class);
+        suite.addTestSuite(IgniteCacheQueryStopOnCancelOrTimeoutDistributedJoinSelfTest.class);
 
         // Other.
         suite.addTestSuite(CacheQueryNewClientSelfTest.class);


[03/50] [abbrv] ignite git commit: Fixes after merge.

Posted by vo...@apache.org.
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/61ab650e
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/61ab650e
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/61ab650e

Branch: refs/heads/ignite-2693
Commit: 61ab650ecd353b4014a30f793090e9dab3519ad5
Parents: b8b9abe
Author: devozerov <vo...@gridgain.com>
Authored: Mon Oct 31 21:33:59 2016 +0300
Committer: thatcoach <pp...@list.ru>
Committed: Mon Oct 31 21:59:30 2016 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/cache/query/SqlQuery.java     |  5 ++---
 .../internal/processors/cache/QueryCursorImpl.java  | 16 +++++++++-------
 .../internal/processors/query/GridQueryCancel.java  |  3 +--
 .../h2/twostep/messages/GridQueryFailResponse.java  |  3 +--
 .../testframework/junits/GridTestKernalContext.java | 13 ++++++++-----
 ...teCacheQueryAbstractDistributedJoinSelfTest.java |  5 ++---
 6 files changed, 23 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/61ab650e/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 3b8fe6d..83e171d 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
@@ -17,15 +17,14 @@
 
 package org.apache.ignite.cache.query;
 
+import java.util.concurrent.TimeUnit;
+import javax.cache.Cache;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
-import javax.cache.Cache;
-import java.util.concurrent.TimeUnit;
-
 /**
  * SQL Query.
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/61ab650e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
index f93a747..5a46d65 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
@@ -17,6 +17,11 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.query.QueryCancelledException;
@@ -24,13 +29,10 @@ import org.apache.ignite.internal.processors.cache.query.QueryCursorEx;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 
-import javax.cache.CacheException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
-
-import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.*;
+import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.CLOSED;
+import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.EXECUTION;
+import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.IDLE;
+import static org.apache.ignite.internal.processors.cache.QueryCursorImpl.State.RESULT_READY;
 
 /**
  * Query cursor implementation.

http://git-wip-us.apache.org/repos/asf/ignite/blob/61ab650e/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java
index 7391f39..d49bcc7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryCancel.java
@@ -17,9 +17,8 @@
 
 package org.apache.ignite.internal.processors.query;
 
-import org.apache.ignite.cache.query.QueryCancelledException;
-
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+import org.apache.ignite.cache.query.QueryCancelledException;
 
 /**
  * Holds query cancel state.

http://git-wip-us.apache.org/repos/asf/ignite/blob/61ab650e/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 7554ae9..0baf6ea 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
@@ -17,14 +17,13 @@
 
 package org.apache.ignite.internal.processors.query.h2.twostep.messages;
 
+import java.nio.ByteBuffer;
 import org.apache.ignite.cache.query.QueryCancelledException;
 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;
 
-import java.nio.ByteBuffer;
-
 /**
  * Error message.
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/61ab650e/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
index 03138c3..f9e2ff4 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
@@ -17,18 +17,21 @@
 
 package org.apache.ignite.testframework.junits;
 
+import java.util.List;
+import java.util.ListIterator;
+import java.util.concurrent.ExecutorService;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.GridComponent;
+import org.apache.ignite.internal.GridKernalContextImpl;
+import org.apache.ignite.internal.GridKernalGatewayImpl;
+import org.apache.ignite.internal.GridLoggerProxy;
+import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 
-import java.util.List;
-import java.util.ListIterator;
-import java.util.concurrent.ExecutorService;
-
 /**
  * Test context.
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/61ab650e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java
index 339e0d3..be3e22d 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryAbstractDistributedJoinSelfTest.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import java.io.Serializable;
+import java.util.Random;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
@@ -32,9 +34,6 @@ 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 java.io.Serializable;
-import java.util.Random;
-
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;


[49/50] [abbrv] ignite git commit: Merge branch 'master' into ignite-2693

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/c5390868/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index 2d0f8d8,297ec68..dc898fc
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@@ -63,7 -63,7 +63,8 @@@ import org.apache.ignite.internal.Async
  import org.apache.ignite.internal.GridKernalContext;
  import org.apache.ignite.internal.IgniteEx;
  import org.apache.ignite.internal.IgniteInternalFuture;
 +import org.apache.ignite.internal.binary.BinaryMarshaller;
+ import org.apache.ignite.internal.binary.BinaryUtils;
  import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
  import org.apache.ignite.internal.processors.cache.query.CacheQuery;
  import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture;

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


[38/50] [abbrv] ignite git commit: .NET: Fix error messages when IgniteConfigurationSection content is missing

Posted by vo...@apache.org.
.NET: Fix error messages when IgniteConfigurationSection content is missing


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

Branch: refs/heads/ignite-2693
Commit: 058ad505547c4d35afd16cc3f962ecfb04dd924c
Parents: d775ad1
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Nov 8 19:38:28 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Wed Nov 9 19:53:52 2016 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.AspNet/Impl/ConfigUtil.cs     |  2 +-
 .../IgniteConfigurationSectionTest.cs           | 20 ++++++++++++++++++++
 .../dotnet/Apache.Ignite.Core.Tests/app.config  |  1 +
 .../Apache.Ignite.Core.Tests/custom_app.config  |  1 +
 .../dotnet/Apache.Ignite.Core/Ignition.cs       | 12 ++++++++++++
 5 files changed, 35 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/058ad505/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs
index fc93c7e..b967518 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Impl/ConfigUtil.cs
@@ -89,7 +89,7 @@ namespace Apache.Ignite.AspNet.Impl
 
                 if (config == null)
                     throw new ConfigurationErrorsException(string.Format(CultureInfo.InvariantCulture,
-                        "{0} with name '{1}' is defined in <configSections>, but not present in configuration", 
+                        "{0} with name '{1}' is defined in <configSections>, but not present in configuration.", 
                         typeof(IgniteConfigurationSection).Name, sectionName));
             }
             else

http://git-wip-us.apache.org/repos/asf/ignite/blob/058ad505/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSectionTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSectionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSectionTest.cs
index 4e10a2b..13bd9a2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSectionTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSectionTest.cs
@@ -78,6 +78,7 @@ namespace Apache.Ignite.Core.Tests
         [Test]
         public void TestIgniteStartError()
         {
+            // Missing section in default file.
             var ex = Assert.Throws<ConfigurationErrorsException>(() =>
                 Ignition.StartFromApplicationConfiguration("igniteConfiguration111"));
 
@@ -85,17 +86,36 @@ namespace Apache.Ignite.Core.Tests
                 ex.Message);
 
 
+            // Missing section body.
+            ex = Assert.Throws<ConfigurationErrorsException>(() =>
+                Ignition.StartFromApplicationConfiguration("igniteConfigurationMissing"));
+
+            Assert.AreEqual("IgniteConfigurationSection with name 'igniteConfigurationMissing' " +
+                            "is defined in <configSections>, but not present in configuration.", ex.Message);
+
+
+            // Missing custom file.
             ex = Assert.Throws<ConfigurationErrorsException>(() =>
                 Ignition.StartFromApplicationConfiguration("igniteConfiguration", "somefile"));
 
             Assert.AreEqual("Specified config file does not exist: somefile", ex.Message);
 
 
+            // Missing section in custom file.
             ex = Assert.Throws<ConfigurationErrorsException>(() =>
                 Ignition.StartFromApplicationConfiguration("igniteConfiguration", "custom_app.config"));
 
             Assert.AreEqual("Could not find IgniteConfigurationSection with name 'igniteConfiguration' " +
                             "in file 'custom_app.config'", ex.Message);
+            
+            
+            // Missing section body in custom file.
+            ex = Assert.Throws<ConfigurationErrorsException>(() =>
+                Ignition.StartFromApplicationConfiguration("igniteConfigurationMissing", "custom_app.config"));
+
+            Assert.AreEqual("IgniteConfigurationSection with name 'igniteConfigurationMissing' in file " +
+                            "'custom_app.config' is defined in <configSections>, but not present in configuration.",
+                ex.Message);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/058ad505/modules/platforms/dotnet/Apache.Ignite.Core.Tests/app.config
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/app.config b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/app.config
index d22bea4..f928ed4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/app.config
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/app.config
@@ -21,6 +21,7 @@
     <configSections>
         <section name="igniteConfiguration" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
         <section name="igniteConfiguration2" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
+        <section name="igniteConfigurationMissing" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
     </configSections>
 
     <runtime>

http://git-wip-us.apache.org/repos/asf/ignite/blob/058ad505/modules/platforms/dotnet/Apache.Ignite.Core.Tests/custom_app.config
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/custom_app.config b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/custom_app.config
index 41ea39e..aec91dc 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/custom_app.config
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/custom_app.config
@@ -20,6 +20,7 @@
 <configuration>
     <configSections>
         <section name="igniteConfiguration3" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
+        <section name="igniteConfigurationMissing" type="Apache.Ignite.Core.IgniteConfigurationSection, Apache.Ignite.Core" />
     </configSections>
 
     <igniteConfiguration3 gridName="myGrid3" localhost="127.0.0.1">

http://git-wip-us.apache.org/repos/asf/ignite/blob/058ad505/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
index 7b023f3..515902f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
@@ -152,6 +152,12 @@ namespace Apache.Ignite.Core
                 throw new ConfigurationErrorsException(string.Format("Could not find {0} with name '{1}'",
                     typeof(IgniteConfigurationSection).Name, sectionName));
 
+            if (section.IgniteConfiguration == null)
+                throw new ConfigurationErrorsException(
+                    string.Format("{0} with name '{1}' is defined in <configSections>, " +
+                                  "but not present in configuration.",
+                        typeof(IgniteConfigurationSection).Name, sectionName));
+
             return Start(section.IgniteConfiguration);
         }
 
@@ -177,6 +183,12 @@ namespace Apache.Ignite.Core
                     string.Format("Could not find {0} with name '{1}' in file '{2}'",
                         typeof(IgniteConfigurationSection).Name, sectionName, configPath));
 
+            if (section.IgniteConfiguration == null)
+                throw new ConfigurationErrorsException(
+                    string.Format("{0} with name '{1}' in file '{2}' is defined in <configSections>, " +
+                                  "but not present in configuration.",
+                        typeof(IgniteConfigurationSection).Name, sectionName, configPath));
+
             return Start(section.IgniteConfiguration);
         }
 


[24/50] [abbrv] ignite git commit: Added new test: IgniteCachePartitionedBackupNodeFailureRecoveryTest.

Posted by vo...@apache.org.
Added new test: IgniteCachePartitionedBackupNodeFailureRecoveryTest.


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

Branch: refs/heads/ignite-2693
Commit: e031498d872a5f1552b26a859b9ee79d3568ff9b
Parents: 45bb1ac
Author: sboikov <sb...@gridgain.com>
Authored: Tue Nov 8 11:21:37 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Nov 8 11:21:37 2016 +0300

----------------------------------------------------------------------
 ...artitionedBackupNodeFailureRecoveryTest.java | 193 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite2.java       |   2 +
 2 files changed, 195 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e031498d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedBackupNodeFailureRecoveryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedBackupNodeFailureRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedBackupNodeFailureRecoveryTest.java
new file mode 100644
index 0000000..6654fd9
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePartitionedBackupNodeFailureRecoveryTest.java
@@ -0,0 +1,193 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements. See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.ignite.internal.processors.cache.distributed.dht;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantLock;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.cache.IgniteCacheAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.PRIMARY;
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+
+/**
+ */
+public class IgniteCachePartitionedBackupNodeFailureRecoveryTest extends IgniteCacheAbstractTest {
+    /** {@inheritDoc}*/
+    @Override protected int gridCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc}*/
+    @Override protected CacheMode cacheMode() {
+        return PARTITIONED;
+    }
+
+    /** {@inheritDoc}*/
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+
+    /** {@inheritDoc}*/
+    @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() {
+        return PRIMARY;
+    }
+
+    /** {@inheritDoc}*/
+    @Override protected NearCacheConfiguration nearConfiguration() {
+        return new NearCacheConfiguration();
+    }
+
+    /** {@inheritDoc}*/
+    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
+        CacheConfiguration ccfg = super.cacheConfiguration(gridName);
+
+        ccfg.setBackups(1);
+        ccfg.setWriteSynchronizationMode(PRIMARY_SYNC);
+        ccfg.setRebalanceMode(SYNC);
+
+        return ccfg;
+    }
+
+    /**
+     * Test stops and restarts backup node.
+     *
+     * @throws Exception If failed.
+     */
+    public void testBackUpFail() throws Exception {
+        final IgniteEx node1 = grid(0);
+        final IgniteEx node2 = grid(1);
+        final IgniteEx node3 = grid(2);
+
+        awaitPartitionMapExchange();
+
+        final IgniteCache<Integer, Integer> cache1 = node1.cache(null);
+
+        Affinity<Integer> aff = node1.affinity(null);
+
+        Integer key0 = null;
+
+        for (int key = 0; key < 10_000; key++) {
+            if (aff.isPrimary(node2.cluster().localNode(), key) && aff.isBackup(node3.cluster().localNode(), key)) {
+                key0 = key;
+
+                break;
+            }
+        }
+
+        assertNotNull(key0);
+
+        cache1.put(key0, 0);
+
+        final AtomicBoolean finished = new AtomicBoolean();
+
+        final ReentrantLock lock = new ReentrantLock();
+
+        final AtomicInteger cntr = new AtomicInteger();
+
+        final Integer finalKey = key0;
+
+        IgniteInternalFuture<Void> primaryFut;
+        IgniteInternalFuture<Void> backupFut;
+
+        try {
+            primaryFut = runAsync(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    while (!finished.get()) {
+                        lock.lock();
+
+                        try {
+                            cache1.invoke(finalKey, new TestEntryProcessor());
+
+                            cntr.getAndIncrement();
+                        }
+                        finally {
+                            lock.unlock();
+                        }
+                    }
+
+                    return null;
+                }
+            });
+
+            backupFut = runAsync(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    while (!finished.get()) {
+                        stopGrid(2);
+
+                        IgniteEx backUp = startGrid(2);
+
+                        IgniteCache<Integer, Integer> cache3 = backUp.cache(null);
+
+                        lock.lock();
+
+                        try {
+                            Integer backUpVal = cache3.localPeek(finalKey);
+
+                            Integer exp = cntr.get();
+
+                            assertEquals(exp, backUpVal);
+                        }
+                        finally {
+                            lock.unlock();
+                        }
+                    }
+                    return null;
+                }
+            });
+
+            Thread.sleep(30_000);
+        }
+        finally {
+            finished.set(true);
+        }
+
+        primaryFut.get();
+        backupFut.get();
+    }
+
+    /**
+     *
+     */
+    static class TestEntryProcessor implements EntryProcessor<Integer, Integer, Void> {
+        /** {@inheritDoc}*/
+        @Override public Void process(MutableEntry<Integer, Integer> entry, Object... args) {
+            Integer v = entry.getValue() + 1;
+
+            entry.setValue(v);
+
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e031498d/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index dc412a9..3fc27de 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -72,6 +72,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePart
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedPreloadEventsSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedTopologyChangeSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedUnloadEventsSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePartitionedBackupNodeFailureRecoveryTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicNearEvictionEventSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicNearMultiNodeSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicNearReadersSelfTest;
@@ -259,6 +260,7 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(CacheEnumOperationsSingleNodeTest.class));
         suite.addTest(new TestSuite(CacheEnumOperationsTest.class));
         suite.addTest(new TestSuite(IgniteCacheIncrementTxTest.class));
+        suite.addTest(new TestSuite(IgniteCachePartitionedBackupNodeFailureRecoveryTest.class));
 
         suite.addTest(new TestSuite(IgniteNoCustomEventsOnNodeStart.class));
 


[41/50] [abbrv] ignite git commit: IGNITE-4207: Updated Flume sink to 1.7.0. - Fixes #1226.

Posted by vo...@apache.org.
IGNITE-4207: Updated Flume sink to 1.7.0. - Fixes #1226.

Signed-off-by: shtykh_roman <rs...@yahoo.com>


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

Branch: refs/heads/ignite-2693
Commit: aacdcba6a2516f5b4867163938445032a82ba7d5
Parents: c6e0061
Author: shtykh_roman <rs...@yahoo.com>
Authored: Thu Nov 10 18:44:31 2016 +0900
Committer: shtykh_roman <rs...@yahoo.com>
Committed: Thu Nov 10 18:44:31 2016 +0900

----------------------------------------------------------------------
 modules/flume/README.txt | 2 +-
 parent/pom.xml           | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/aacdcba6/modules/flume/README.txt
----------------------------------------------------------------------
diff --git a/modules/flume/README.txt b/modules/flume/README.txt
index bf7e0ff..adcd021 100644
--- a/modules/flume/README.txt
+++ b/modules/flume/README.txt
@@ -2,7 +2,7 @@ Apache Ignite Flume Sink Module
 -------------------------------
 
 IgniteSink is a Flume sink that extracts Events from an associated Flume channel and injects into an Ignite cache.
-Flume 1.6.0 is supported.
+Flume 1.7.0 is supported.
 
 IgniteSink, which can be found in 'optional/ignite-flume', and its dependencies have to be included in the agent's classpath,
 as described in the following subsection, before starting the Flume agent.

http://git-wip-us.apache.org/repos/asf/ignite/blob/aacdcba6/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 618bb90..682efa2 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -65,7 +65,7 @@
         <easymock.version>3.4</easymock.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>
+        <flume.ng.version>1.7.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>


[09/50] [abbrv] ignite git commit: IGNITE-3675 .NET: Use separate caches for different entities in QueryExample.

Posted by vo...@apache.org.
IGNITE-3675 .NET: Use separate caches for different entities in QueryExample.


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

Branch: refs/heads/ignite-2693
Commit: 390c8d59cf8d44d9703e5fbf8d9c0f8028a4a172
Parents: d7dee52
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Thu Aug 11 16:18:14 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Wed Nov 2 11:16:33 2016 +0300

----------------------------------------------------------------------
 .../Datagrid/LinqExample.cs                     | 38 ++++++++++++-------
 .../Datagrid/QueryExample.cs                    | 39 +++++++++++++-------
 2 files changed, 51 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/390c8d59/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs
index 2223600..848d8f5 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/LinqExample.cs
@@ -43,8 +43,11 @@ namespace Apache.Ignite.Examples.Datagrid
     /// </summary>
     public class LinqExample
     {
-        /// <summary>Cache name.</summary>
-        private const string CacheName = "dotnet_cache_query";
+        /// <summary>Organization cache name.</summary>
+        private const string OrganizationCacheName = "dotnet_cache_query_organization";
+
+        /// <summary>Employee cache name.</summary>
+        private const string EmployeeCacheName = "dotnet_cache_query_employee";
 
         [STAThread]
         public static void Main()
@@ -54,25 +57,27 @@ namespace Apache.Ignite.Examples.Datagrid
                 Console.WriteLine();
                 Console.WriteLine(">>> Cache LINQ example started.");
 
-                var cache = ignite.GetOrCreateCache<object, object>(new CacheConfiguration
+                var employeeCache = ignite.GetOrCreateCache<EmployeeKey, Employee>(new CacheConfiguration
                 {
-                    Name = CacheName,
+                    Name = EmployeeCacheName,
                     QueryEntities = new[]
                     {
-                        new QueryEntity(typeof(int), typeof(Organization)),
                         new QueryEntity(typeof(EmployeeKey), typeof(Employee))
                     }
                 });
 
-                // Clean up caches on all nodes before run.
-                cache.Clear();
+                var organizationCache = ignite.GetOrCreateCache<int, Organization>(new CacheConfiguration
+                {
+                    Name = OrganizationCacheName,
+                    QueryEntities = new[]
+                    {
+                        new QueryEntity(typeof(int), typeof(Organization))
+                    }
+                });
 
                 // Populate cache with sample data entries.
-                PopulateCache(cache);
-
-                // Create cache that will work with specific types.
-                var employeeCache = ignite.GetCache<EmployeeKey, Employee>(CacheName);
-                var organizationCache = ignite.GetCache<int, Organization>(CacheName);
+                PopulateCache(employeeCache);
+                PopulateCache(organizationCache);
 
                 // Run SQL query example.
                 QueryExample(employeeCache);
@@ -177,7 +182,7 @@ namespace Apache.Ignite.Examples.Datagrid
         /// Populate cache with data for this example.
         /// </summary>
         /// <param name="cache">Cache.</param>
-        private static void PopulateCache(ICache<object, object> cache)
+        private static void PopulateCache(ICache<int, Organization> cache)
         {
             cache.Put(1, new Organization(
                 "Apache",
@@ -192,7 +197,14 @@ namespace Apache.Ignite.Examples.Datagrid
                 OrganizationType.Private,
                 DateTime.Now
             ));
+        }
 
+        /// <summary>
+        /// Populate cache with data for this example.
+        /// </summary>
+        /// <param name="cache">Cache.</param>
+        private static void PopulateCache(ICache<EmployeeKey, Employee> cache)
+        {
             cache.Put(new EmployeeKey(1, 1), new Employee(
                 "James Wilson",
                 12500,

http://git-wip-us.apache.org/repos/asf/ignite/blob/390c8d59/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs
index ccd6fd9..8b5e6f3 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/QueryExample.cs
@@ -43,8 +43,11 @@ namespace Apache.Ignite.Examples.Datagrid
     /// </summary>
     public class QueryExample
     {
-        /// <summary>Cache name.</summary>
-        private const string CacheName = "dotnet_cache_query";
+        /// <summary>Organization cache name.</summary>
+        private const string OrganizationCacheName = "dotnet_cache_query_organization";
+
+        /// <summary>Employee cache name.</summary>
+        private const string EmployeeCacheName = "dotnet_cache_query_employee";
 
         [STAThread]
         public static void Main()
@@ -54,24 +57,27 @@ namespace Apache.Ignite.Examples.Datagrid
                 Console.WriteLine();
                 Console.WriteLine(">>> Cache query example started.");
 
-                var cache = ignite.GetOrCreateCache<object, object>(new CacheConfiguration
+                var employeeCache = ignite.GetOrCreateCache<EmployeeKey, Employee>(new CacheConfiguration
                 {
-                    Name = CacheName,
+                    Name = EmployeeCacheName,
                     QueryEntities = new[]
                     {
-                        new QueryEntity(typeof(int), typeof(Organization)),
                         new QueryEntity(typeof(EmployeeKey), typeof(Employee))
                     }
                 });
 
-                // Clean up caches on all nodes before run.
-                cache.Clear();
+                var organizationCache = ignite.GetOrCreateCache<int, Organization>(new CacheConfiguration
+                {
+                    Name = OrganizationCacheName,
+                    QueryEntities = new[]
+                    {
+                        new QueryEntity(typeof(int), typeof(Organization))
+                    }
+                });
 
                 // Populate cache with sample data entries.
-                PopulateCache(cache);
-
-                // Create cache that will work with specific types.
-                var employeeCache = ignite.GetCache<EmployeeKey, Employee>(CacheName);
+                PopulateCache(employeeCache);
+                PopulateCache(organizationCache);
 
                 // Run SQL query example.
                 SqlQueryExample(employeeCache);
@@ -119,7 +125,7 @@ namespace Apache.Ignite.Examples.Datagrid
             const string orgName = "Apache";
 
             var qry = cache.Query(new SqlQuery("Employee",
-                "from Employee, Organization " +
+                "from Employee, \"dotnet_cache_query_organization\".Organization " +
                 "where Employee.organizationId = Organization._key and Organization.name = ?", orgName));
 
             Console.WriteLine();
@@ -163,7 +169,7 @@ namespace Apache.Ignite.Examples.Datagrid
         /// Populate cache with data for this example.
         /// </summary>
         /// <param name="cache">Cache.</param>
-        private static void PopulateCache(ICache<object, object> cache)
+        private static void PopulateCache(ICache<int, Organization> cache)
         {
             cache.Put(1, new Organization(
                 "Apache",
@@ -178,7 +184,14 @@ namespace Apache.Ignite.Examples.Datagrid
                 OrganizationType.Private,
                 DateTime.Now
             ));
+        }
 
+        /// <summary>
+        /// Populate cache with data for this example.
+        /// </summary>
+        /// <param name="cache">Cache.</param>
+        private static void PopulateCache(ICache<EmployeeKey, Employee> cache)
+        {
             cache.Put(new EmployeeKey(1, 1), new Employee(
                 "James Wilson",
                 12500,


[48/50] [abbrv] ignite git commit: .NET: Fix EntityFramework project settings

Posted by vo...@apache.org.
.NET: Fix EntityFramework project settings


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

Branch: refs/heads/ignite-2693
Commit: dd20d943a2d8052e29f747f3bfbc6df0f1d535dd
Parents: 3aae5cb
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Nov 14 16:53:58 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Nov 14 16:53:58 2016 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.EntityFramework.csproj                         | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dd20d943/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.csproj b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.csproj
index 8b3c651..7082182 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.csproj
@@ -23,14 +23,16 @@
     <RunCodeAnalysis>true</RunCodeAnalysis>
     <CodeAnalysisRuleSet>AllRules.ruleset</CodeAnalysisRuleSet>
     <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
+    <DocumentationFile>bin\Debug\Apache.Ignite.EntityFramework.XML</DocumentationFile>
   </PropertyGroup>
   <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release|AnyCPU' ">
     <DebugType>pdbonly</DebugType>
     <Optimize>true</Optimize>
     <OutputPath>bin\Release\</OutputPath>
-    <DefineConstants>TRACE</DefineConstants>
     <ErrorReport>prompt</ErrorReport>
     <WarningLevel>4</WarningLevel>
+    <DebugType>none</DebugType>
+	<DocumentationFile>bin\Release\Apache.Ignite.EntityFramework.XML</DocumentationFile>
   </PropertyGroup>
   <PropertyGroup>
     <SignAssembly>true</SignAssembly>