You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by yz...@apache.org on 2016/12/28 14:04:27 UTC

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

Repository: ignite
Updated Branches:
  refs/heads/ignite-comm-balance-master 855399acc -> b8eb27f60


Merge remote-tracking branch 'remotes/community/ignite-1.7.4' into ignite-1.8.2

# Conflicts:
#	modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
#	modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java
#	modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
#	modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
#	modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
#	modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
#	modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 64247b9228451e46abb8029e09c7fc6ed4e16d2d
Parents: 147277d 8dd4ada
Author: sboikov <sb...@gridgain.com>
Authored: Mon Dec 19 15:54:39 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Dec 19 15:54:39 2016 +0300

----------------------------------------------------------------------
 .../store/cassandra/CassandraCacheStore.java    |   9 +-
 .../store/cassandra/datasource/DataSource.java  |   9 +
 .../rest/RestProcessorMultiStartSelfTest.java   |  48 +-
 .../java/org/apache/ignite/IgniteServices.java  |  16 +
 .../apache/ignite/IgniteSystemProperties.java   |   6 +
 .../rendezvous/RendezvousAffinityFunction.java  |  80 ++-
 .../ignite/cache/store/CacheStoreAdapter.java   |   6 +
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |  19 +-
 .../store/jdbc/JdbcTypesDefaultTransformer.java | 112 ++--
 .../apache/ignite/internal/IgniteKernal.java    |  28 +-
 .../ignite/internal/IgniteServicesImpl.java     |   9 +-
 .../internal/binary/BinaryClassDescriptor.java  |  12 +-
 .../ignite/internal/binary/BinaryUtils.java     |  10 +-
 .../binary/builder/BinaryObjectBuilderImpl.java |  11 +-
 .../discovery/GridDiscoveryManager.java         | 118 +---
 .../affinity/GridAffinityProcessor.java         |   2 +-
 .../processors/cache/CacheLockCandidates.java   |  42 ++
 .../cache/CacheLockCandidatesList.java          |  71 +++
 .../cache/CacheStoreBalancingWrapper.java       |   6 +
 .../processors/cache/GridCacheAdapter.java      |   8 +-
 .../processors/cache/GridCacheEntryEx.java      |   3 +-
 .../cache/GridCacheLoaderWriterStore.java       |   6 +
 .../processors/cache/GridCacheMapEntry.java     | 117 +++-
 .../processors/cache/GridCacheMvcc.java         | 376 +++++++----
 .../processors/cache/GridCacheMvccCallback.java |   4 +-
 .../cache/GridCacheMvccCandidate.java           |  80 +--
 .../processors/cache/GridCacheMvccManager.java  |  19 +-
 .../GridCachePartitionExchangeManager.java      | 157 ++---
 .../processors/cache/GridCachePreloader.java    |  11 +-
 .../cache/GridCachePreloaderAdapter.java        |   5 +-
 .../processors/cache/GridCacheProcessor.java    |  10 +-
 .../processors/cache/GridCacheUtils.java        |  17 -
 .../binary/CacheObjectBinaryProcessorImpl.java  |   3 +-
 .../CacheDataStructuresManager.java             |   6 +-
 .../distributed/GridDistributedCacheEntry.java  | 303 +++------
 .../dht/GridClientPartitionTopology.java        | 120 ++--
 .../distributed/dht/GridDhtCacheEntry.java      |  32 +-
 .../distributed/dht/GridDhtLockFuture.java      |  34 +-
 .../dht/GridDhtPartitionTopology.java           |  28 +-
 .../dht/GridDhtPartitionTopologyImpl.java       | 284 +++++----
 .../dht/GridDhtTransactionalCacheAdapter.java   |   1 -
 .../distributed/dht/GridDhtTxPrepareFuture.java |   5 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   8 +-
 .../dht/preloader/GridDhtPartitionDemander.java | 230 ++++---
 .../dht/preloader/GridDhtPartitionFullMap.java  |  18 +-
 .../GridDhtPartitionsExchangeFuture.java        |  56 +-
 .../dht/preloader/GridDhtPreloader.java         |   9 +-
 .../distributed/near/GridNearCacheEntry.java    |  44 +-
 .../distributed/near/GridNearLockFuture.java    |   3 +-
 .../near/GridNearTransactionalCache.java        |   5 +-
 .../cache/local/GridLocalCacheEntry.java        | 173 ++----
 .../cache/local/GridLocalLockFuture.java        |   2 +-
 .../cache/query/GridCacheQueryManager.java      |  22 +-
 .../cache/transactions/IgniteTxHandler.java     |   2 +-
 .../cache/transactions/IgniteTxManager.java     |   5 +-
 .../closure/GridClosureProcessor.java           |  31 +-
 .../internal/processors/job/GridJobWorker.java  |  76 ++-
 .../processors/odbc/OdbcRequestHandler.java     |  14 +-
 .../platform/PlatformContextImpl.java           |   2 +-
 .../dotnet/PlatformDotNetCacheStore.java        |  11 +
 .../platform/services/PlatformServices.java     |   2 +-
 .../platform/utils/PlatformUtils.java           |  28 +
 .../processors/rest/GridRestProcessor.java      |  15 +
 .../service/GridServiceProcessor.java           |  15 +-
 .../processors/service/GridServiceProxy.java    |  18 +-
 .../processors/task/GridTaskWorker.java         |   7 +
 .../internal/visor/query/VisorQueryJob.java     |   2 +-
 .../ignite/marshaller/jdk/JdkMarshaller.java    |   4 +-
 .../optimized/OptimizedMarshaller.java          |   8 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  41 +-
 .../tcp/internal/TcpDiscoveryStatistics.java    |   4 +
 .../resources/META-INF/classnames.properties    |  86 ++-
 .../AbstractAffinityFunctionSelfTest.java       |   2 +-
 .../jdbc/JdbcTypesDefaultTransformerTest.java   | 283 +++++++++
 .../IgniteComputeTopologyExceptionTest.java     |   5 +-
 .../binary/BinaryMarshallerSelfTest.java        |  66 ++
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   2 +-
 .../CacheSerializableTransactionsTest.java      | 604 +++++++++++++++++-
 .../cache/GridCacheMvccFlagsTest.java           |   8 +-
 .../cache/GridCacheMvccPartitionedSelfTest.java | 334 ++++++++--
 .../processors/cache/GridCacheMvccSelfTest.java | 212 +++----
 .../GridCachePartitionedAffinitySpreadTest.java |   7 +-
 .../processors/cache/GridCacheTestEntryEx.java  |  77 +--
 ...heapCacheMetricsForClusterGroupSelfTest.java | 141 +++++
 .../cache/OffheapCacheOnClientsTest.java        | 143 +++++
 .../distributed/dht/GridCacheDhtTestUtils.java  | 232 -------
 .../GridCacheRebalancingSyncSelfTest.java       |   2 +
 .../CacheOffHeapAndSwapMetricsSelfTest.java     | 621 -------------------
 ...LocalCacheOffHeapAndSwapMetricsSelfTest.java | 621 +++++++++++++++++++
 .../closure/GridClosureSerializationTest.java   | 177 ++++++
 ...gniteServiceProxyTimeoutInitializedTest.java | 284 +++++++++
 .../loadtests/hashmap/GridHashMapLoadTest.java  |   7 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 .../IgniteCacheMetricsSelfTestSuite.java        |   6 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 .../testsuites/IgniteCacheTestSuite2.java       |   2 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |   2 +
 .../resources/META-INF/classnames.properties    | 114 ++++
 .../processors/query/h2/IgniteH2Indexing.java   |  77 ++-
 .../h2/twostep/GridReduceQueryExecutor.java     |  14 +-
 ...niteCachePartitionedFieldsQuerySelfTest.java |  25 +
 101 files changed, 4783 insertions(+), 2473 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
----------------------------------------------------------------------
diff --cc modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
index 9058837,0000000..b4bed0d
mode 100644,000000..100644
--- a/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
+++ b/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
@@@ -1,519 -1,0 +1,522 @@@
 +/*
 + * 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.cache.store.cassandra;
 +
 +import com.datastax.driver.core.BoundStatement;
 +import com.datastax.driver.core.PreparedStatement;
 +import com.datastax.driver.core.Row;
 +import java.util.ArrayList;
 +import java.util.Collection;
 +import java.util.LinkedList;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.HashMap;
 +import java.util.concurrent.ExecutorService;
 +import java.util.concurrent.Executors;
 +import java.util.concurrent.Future;
 +import javax.cache.Cache;
 +import javax.cache.integration.CacheLoaderException;
 +import javax.cache.integration.CacheWriterException;
 +import org.apache.ignite.IgniteCheckedException;
 +import org.apache.ignite.IgniteLogger;
 +import org.apache.ignite.cache.store.CacheStore;
 +import org.apache.ignite.cache.store.CacheStoreSession;
 +import org.apache.ignite.cache.store.cassandra.datasource.DataSource;
 +import org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings;
 +import org.apache.ignite.cache.store.cassandra.persistence.PersistenceController;
 +import org.apache.ignite.cache.store.cassandra.session.CassandraSession;
 +import org.apache.ignite.cache.store.cassandra.session.ExecutionAssistant;
 +import org.apache.ignite.cache.store.cassandra.session.GenericBatchExecutionAssistant;
 +import org.apache.ignite.cache.store.cassandra.session.LoadCacheCustomQueryWorker;
- import org.apache.ignite.cache.store.cassandra.session.transaction.DeleteMutation;
- import org.apache.ignite.cache.store.cassandra.session.transaction.Mutation;
- import org.apache.ignite.cache.store.cassandra.session.transaction.WriteMutation;
++import org.apache.ignite.internal.util.typedef.internal.S;
 +import org.apache.ignite.internal.util.typedef.internal.U;
 +import org.apache.ignite.lang.IgniteBiInClosure;
 +import org.apache.ignite.logger.NullLogger;
 +import org.apache.ignite.resources.CacheStoreSessionResource;
 +import org.apache.ignite.resources.LoggerResource;
 +
 +/**
 + * Implementation of {@link CacheStore} backed by Cassandra database.
 + *
 + * @param <K> Ignite cache key type.
 + * @param <V> Ignite cache value type.
 + */
 +public class CassandraCacheStore<K, V> implements CacheStore<K, V> {
 +    /** Buffer to store mutations performed withing transaction. */
 +    private static final String TRANSACTION_BUFFER = "CASSANDRA_TRANSACTION_BUFFER";
 +
 +    /** Auto-injected store session. */
 +    @SuppressWarnings("unused")
 +    @CacheStoreSessionResource
 +    private CacheStoreSession storeSes;
 +
 +    /** Auto-injected logger instance. */
 +    @SuppressWarnings("unused")
 +    @LoggerResource
 +    private IgniteLogger log;
 +
 +    /** Cassandra data source. */
 +    private DataSource dataSrc;
 +
 +    /** Max workers thread count. These threads are responsible for load cache. */
 +    private int maxPoolSize = Runtime.getRuntime().availableProcessors();
 +
 +    /** Controller component responsible for serialization logic. */
 +    private final PersistenceController controller;
 +
 +    /**
 +     * Store constructor.
 +     *
 +     * @param dataSrc Data source.
 +     * @param settings Persistence settings for Ignite key and value objects.
 +     * @param maxPoolSize Max workers thread count.
 +     */
 +    public CassandraCacheStore(DataSource dataSrc, KeyValuePersistenceSettings settings, int maxPoolSize) {
 +        this.dataSrc = dataSrc;
 +        this.controller = new PersistenceController(settings);
 +        this.maxPoolSize = maxPoolSize;
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public void loadCache(IgniteBiInClosure<K, V> clo, Object... args) throws CacheLoaderException {
 +        if (clo == null)
 +            return;
 +
 +        if (args == null || args.length == 0)
 +            args = new String[] {"select * from " + controller.getPersistenceSettings().getKeyspace() + "." + cassandraTable() + ";"};
 +
 +        ExecutorService pool = null;
 +
 +        Collection<Future<?>> futs = new ArrayList<>(args.length);
 +
 +        try {
 +            pool = Executors.newFixedThreadPool(maxPoolSize);
 +
 +            CassandraSession ses = getCassandraSession();
 +
 +            for (Object obj : args) {
 +                if (obj == null || !(obj instanceof String) || !((String)obj).trim().toLowerCase().startsWith("select"))
 +                    continue;
 +
 +                futs.add(pool.submit(new LoadCacheCustomQueryWorker<>(ses, (String) obj, controller, log, clo)));
 +            }
 +
 +            for (Future<?> fut : futs)
 +                U.get(fut);
 +
 +            if (log != null && log.isDebugEnabled() && storeSes != null)
 +                log.debug("Cache loaded from db: " + storeSes.cacheName());
 +        }
 +        catch (IgniteCheckedException e) {
 +            if (storeSes != null)
 +                throw new CacheLoaderException("Failed to load Ignite cache: " + storeSes.cacheName(), e.getCause());
 +            else
 +                throw new CacheLoaderException("Failed to load cache", e.getCause());
 +        }
 +        finally {
 +            U.shutdownNow(getClass(), pool, log);
 +        }
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public void sessionEnd(boolean commit) throws CacheWriterException {
 +        if (!storeSes.isWithinTransaction())
 +            return;
 +
 +        List<Mutation> mutations = mutations();
 +        if (mutations == null || mutations.isEmpty())
 +            return;
 +
 +        CassandraSession ses = getCassandraSession();
 +
 +        try {
 +            ses.execute(mutations);
 +        }
 +        finally {
 +            mutations.clear();
 +            U.closeQuiet(ses);
 +        }
 +    }
 +
 +    /** {@inheritDoc} */
 +    @SuppressWarnings({"unchecked"})
 +    @Override public V load(final K key) throws CacheLoaderException {
 +        if (key == null)
 +            return null;
 +
 +        CassandraSession ses = getCassandraSession();
 +
 +        try {
 +            return ses.execute(new ExecutionAssistant<V>() {
 +                /** {@inheritDoc} */
 +                @Override public boolean tableExistenceRequired() {
 +                    return false;
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String getTable() {
 +                    return cassandraTable();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String getStatement() {
 +                    return controller.getLoadStatement(cassandraTable(), false);
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public BoundStatement bindStatement(PreparedStatement statement) {
 +                    return controller.bindKey(statement, key);
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public KeyValuePersistenceSettings getPersistenceSettings() {
 +                    return controller.getPersistenceSettings();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String operationName() {
 +                    return "READ";
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public V process(Row row) {
 +                    return row == null ? null : (V)controller.buildValueObject(row);
 +                }
 +            });
 +        }
 +        finally {
 +            U.closeQuiet(ses);
 +        }
 +    }
 +
 +    /** {@inheritDoc} */
 +    @SuppressWarnings("unchecked")
 +    @Override public Map<K, V> loadAll(Iterable<? extends K> keys) throws CacheLoaderException {
 +        if (keys == null || !keys.iterator().hasNext())
 +            return new HashMap<>();
 +
 +        CassandraSession ses = getCassandraSession();
 +
 +        try {
 +            return ses.execute(new GenericBatchExecutionAssistant<Map<K, V>, K>() {
 +                private Map<K, V> data = new HashMap<>();
 +
 +                /** {@inheritDoc} */
 +                @Override public String getTable() {
 +                    return cassandraTable();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String getStatement() {
 +                    return controller.getLoadStatement(cassandraTable(), true);
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override  public BoundStatement bindStatement(PreparedStatement statement, K key) {
 +                    return controller.bindKey(statement, key);
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public KeyValuePersistenceSettings getPersistenceSettings() {
 +                    return controller.getPersistenceSettings();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String operationName() {
 +                    return "BULK_READ";
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public Map<K, V> processedData() {
 +                    return data;
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override protected void process(Row row) {
 +                    data.put((K)controller.buildKeyObject(row), (V)controller.buildValueObject(row));
 +                }
 +            }, keys);
 +        }
 +        finally {
 +            U.closeQuiet(ses);
 +        }
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public void write(final Cache.Entry<? extends K, ? extends V> entry) throws CacheWriterException {
 +        if (entry == null || entry.getKey() == null)
 +            return;
 +
 +        if (storeSes.isWithinTransaction()) {
 +            accumulate(new WriteMutation(entry, cassandraTable(), controller));
 +            return;
 +        }
 +
 +        CassandraSession ses = getCassandraSession();
 +
 +        try {
 +            ses.execute(new ExecutionAssistant<Void>() {
 +                /** {@inheritDoc} */
 +                @Override public boolean tableExistenceRequired() {
 +                    return true;
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String getTable() {
 +                    return cassandraTable();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String getStatement() {
 +                    return controller.getWriteStatement(cassandraTable());
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public BoundStatement bindStatement(PreparedStatement statement) {
 +                    return controller.bindKeyValue(statement, entry.getKey(), entry.getValue());
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public KeyValuePersistenceSettings getPersistenceSettings() {
 +                    return controller.getPersistenceSettings();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String operationName() {
 +                    return "WRITE";
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public Void process(Row row) {
 +                    return null;
 +                }
 +            });
 +        }
 +        finally {
 +            U.closeQuiet(ses);
 +        }
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public void writeAll(Collection<Cache.Entry<? extends K, ? extends V>> entries) throws CacheWriterException {
 +        if (entries == null || entries.isEmpty())
 +            return;
 +
 +        if (storeSes.isWithinTransaction()) {
 +            for (Cache.Entry<?, ?> entry : entries)
 +                accumulate(new WriteMutation(entry, cassandraTable(), controller));
 +
 +            return;
 +        }
 +
 +        CassandraSession ses = getCassandraSession();
 +
 +        try {
 +            ses.execute(new GenericBatchExecutionAssistant<Void, Cache.Entry<? extends K, ? extends V>>() {
 +                /** {@inheritDoc} */
 +                @Override public String getTable() {
 +                    return cassandraTable();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String getStatement() {
 +                    return controller.getWriteStatement(cassandraTable());
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public BoundStatement bindStatement(PreparedStatement statement,
 +                    Cache.Entry<? extends K, ? extends V> entry) {
 +                    return controller.bindKeyValue(statement, entry.getKey(), entry.getValue());
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public KeyValuePersistenceSettings getPersistenceSettings() {
 +                    return controller.getPersistenceSettings();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String operationName() {
 +                    return "BULK_WRITE";
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public boolean tableExistenceRequired() {
 +                    return true;
 +                }
 +            }, entries);
 +        }
 +        finally {
 +            U.closeQuiet(ses);
 +        }
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public void delete(final Object key) throws CacheWriterException {
 +        if (key == null)
 +            return;
 +
 +        if (storeSes.isWithinTransaction()) {
 +            accumulate(new DeleteMutation(key, cassandraTable(), controller));
 +            return;
 +        }
 +
 +        CassandraSession ses = getCassandraSession();
 +
 +        try {
 +            ses.execute(new ExecutionAssistant<Void>() {
 +                /** {@inheritDoc} */
 +                @Override public boolean tableExistenceRequired() {
 +                    return false;
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String getTable() {
 +                    return cassandraTable();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String getStatement() {
 +                    return controller.getDeleteStatement(cassandraTable());
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public BoundStatement bindStatement(PreparedStatement statement) {
 +                    return controller.bindKey(statement, key);
 +                }
 +
 +
 +                /** {@inheritDoc} */
 +                @Override public KeyValuePersistenceSettings getPersistenceSettings() {
 +                    return controller.getPersistenceSettings();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String operationName() {
 +                    return "DELETE";
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public Void process(Row row) {
 +                    return null;
 +                }
 +            });
 +        }
 +        finally {
 +            U.closeQuiet(ses);
 +        }
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public void deleteAll(Collection<?> keys) throws CacheWriterException {
 +        if (keys == null || keys.isEmpty())
 +            return;
 +
 +        if (storeSes.isWithinTransaction()) {
 +            for (Object key : keys)
 +                accumulate(new DeleteMutation(key, cassandraTable(), controller));
 +
 +            return;
 +        }
 +
 +        CassandraSession ses = getCassandraSession();
 +
 +        try {
 +            ses.execute(new GenericBatchExecutionAssistant<Void, Object>() {
 +                /** {@inheritDoc} */
 +                @Override public String getTable() {
 +                    return cassandraTable();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String getStatement() {
 +                    return controller.getDeleteStatement(cassandraTable());
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public BoundStatement bindStatement(PreparedStatement statement, Object key) {
 +                    return controller.bindKey(statement, key);
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public KeyValuePersistenceSettings getPersistenceSettings() {
 +                    return controller.getPersistenceSettings();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String operationName() {
 +                    return "BULK_DELETE";
 +                }
 +            }, keys);
 +        }
 +        finally {
 +            U.closeQuiet(ses);
 +        }
 +    }
 +
 +    /**
 +     * Gets Cassandra session wrapper or creates new if it doesn't exist.
 +     * This wrapper hides all the low-level Cassandra interaction details by providing only high-level methods.
 +     *
 +     * @return Cassandra session wrapper.
 +     */
 +    private CassandraSession getCassandraSession() {
 +        return dataSrc.session(log != null ? log : new NullLogger());
 +    }
 +
 +    /**
 +     * Returns table name to use for all Cassandra based operations (READ/WRITE/DELETE).
 +     *
 +     * @return Table name.
 +     */
 +    private String cassandraTable() {
 +        return controller.getPersistenceSettings().getTable() != null ?
 +            controller.getPersistenceSettings().getTable() : storeSes.cacheName().trim().toLowerCase();
 +    }
 +
 +    /**
 +     * Accumulates mutation in the transaction buffer.
 +     *
 +     * @param mutation Mutation operation.
 +     */
 +    private void accumulate(Mutation mutation) {
 +        //noinspection unchecked
 +        List<Mutation> mutations = (List<Mutation>)storeSes.properties().get(TRANSACTION_BUFFER);
 +
 +        if (mutations == null) {
 +            mutations = new LinkedList<>();
 +            storeSes.properties().put(TRANSACTION_BUFFER, mutations);
 +        }
 +
 +        mutations.add(mutation);
 +    }
 +
 +    /**
 +     * Returns all the mutations performed withing transaction.
 +     *
 +     * @return Mutations
 +     */
 +    private List<Mutation> mutations() {
 +        //noinspection unchecked
 +        return (List<Mutation>)storeSes.properties().get(TRANSACTION_BUFFER);
 +    }
++
++    /** {@inheritDoc} */
++    @Override public String toString() {
++        return S.toString(CassandraCacheStore.class, this);
++    }
 +}

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java
----------------------------------------------------------------------
diff --cc modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java
index f582aac,0000000..1ba3c7d
mode 100644,000000..100644
--- a/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java
+++ b/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java
@@@ -1,647 -1,0 +1,656 @@@
 +/*
 + * 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.cache.store.cassandra.datasource;
 +
 +import com.datastax.driver.core.AuthProvider;
 +import com.datastax.driver.core.Cluster;
 +import com.datastax.driver.core.ConsistencyLevel;
 +import com.datastax.driver.core.NettyOptions;
 +import com.datastax.driver.core.PoolingOptions;
 +import com.datastax.driver.core.ProtocolOptions;
 +import com.datastax.driver.core.ProtocolVersion;
 +import com.datastax.driver.core.SSLOptions;
 +import com.datastax.driver.core.SocketOptions;
 +import com.datastax.driver.core.policies.AddressTranslator;
 +import com.datastax.driver.core.policies.LoadBalancingPolicy;
 +import com.datastax.driver.core.policies.ReconnectionPolicy;
 +import com.datastax.driver.core.policies.RetryPolicy;
 +import com.datastax.driver.core.policies.SpeculativeExecutionPolicy;
 +
 +import java.io.Externalizable;
 +import java.io.IOException;
 +import java.io.ObjectInput;
 +import java.io.ObjectOutput;
 +import java.io.Serializable;
 +import java.net.InetAddress;
 +import java.net.InetSocketAddress;
 +import java.util.LinkedList;
 +import java.util.List;
 +import java.util.UUID;
 +
 +import org.apache.ignite.IgniteException;
 +import org.apache.ignite.IgniteLogger;
 +import org.apache.ignite.cache.store.cassandra.session.CassandraSession;
 +import org.apache.ignite.cache.store.cassandra.session.CassandraSessionImpl;
 +import org.apache.ignite.internal.util.typedef.internal.U;
++import org.apache.ignite.internal.util.tostring.GridToStringExclude;
++import org.apache.ignite.internal.util.typedef.internal.S;
 +
 +/**
 + * Data source abstraction to specify configuration of the Cassandra session to be used.
 + */
 +public class DataSource implements Externalizable {
 +    /** */
 +    private static final long serialVersionUID = 0L;
 +
 +    /**
 +     * Null object, used as a replacement for those Cassandra connection options which
 +     * don't support serialization (RetryPolicy, LoadBalancingPolicy and etc).
 +     */
 +    private static final UUID NULL_OBJECT = UUID.fromString("45ffae47-3193-5910-84a2-048fe65735d9");
 +
 +    /** Number of rows to immediately fetch in CQL statement execution. */
 +    private Integer fetchSize;
 +
 +    /** Consistency level for READ operations. */
 +    private ConsistencyLevel readConsistency;
 +
 +    /** Consistency level for WRITE operations. */
 +    private ConsistencyLevel writeConsistency;
 +
 +    /** Username to use for authentication. */
++    @GridToStringExclude
 +    private String user;
 +
 +    /** Password to use for authentication. */
++    @GridToStringExclude
 +    private String pwd;
 +
 +    /** Port to use for Cassandra connection. */
 +    private Integer port;
 +
 +    /** List of contact points to connect to Cassandra cluster. */
 +    private List<InetAddress> contactPoints;
 +
 +    /** List of contact points with ports to connect to Cassandra cluster. */
 +    private List<InetSocketAddress> contactPointsWithPorts;
 +
 +    /** Maximum time to wait for schema agreement before returning from a DDL query. */
 +    private Integer maxSchemaAgreementWaitSeconds;
 +
 +    /** The native protocol version to use. */
 +    private Integer protoVer;
 +
 +    /** Compression to use for the transport. */
 +    private String compression;
 +
 +    /** Use SSL for communications with Cassandra. */
 +    private Boolean useSSL;
 +
 +    /** Enables metrics collection. */
 +    private Boolean collectMetrix;
 +
 +    /** Enables JMX reporting of the metrics. */
 +    private Boolean jmxReporting;
 +
 +    /** Credentials to use for authentication. */
 +    private Credentials creds;
 +
 +    /** Load balancing policy to use. */
 +    private LoadBalancingPolicy loadBalancingPlc;
 +
 +    /** Reconnection policy to use. */
 +    private ReconnectionPolicy reconnectionPlc;
 +
 +    /** Retry policy to use. */
 +    private RetryPolicy retryPlc;
 +
 +    /** Address translator to use. */
 +    private AddressTranslator addrTranslator;
 +
 +    /** Speculative execution policy to use. */
 +    private SpeculativeExecutionPolicy speculativeExecutionPlc;
 +
 +    /** Authentication provider to use. */
 +    private AuthProvider authProvider;
 +
 +    /** SSL options to use. */
 +    private SSLOptions sslOptions;
 +
 +    /** Connection pooling options to use. */
 +    private PoolingOptions poolingOptions;
 +
 +    /** Socket options to use. */
 +    private SocketOptions sockOptions;
 +
 +    /** Netty options to use for connection. */
 +    private NettyOptions nettyOptions;
 +
 +    /** Cassandra session wrapper instance. */
 +    private volatile CassandraSession ses;
 +
 +    /**
 +     * Sets user name to use for authentication.
 +     *
 +     * @param user user name
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setUser(String user) {
 +        this.user = user;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets password to use for authentication.
 +     *
 +     * @param pwd password
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setPassword(String pwd) {
 +        this.pwd = pwd;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets port to use for Cassandra connection.
 +     *
 +     * @param port port
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setPort(int port) {
 +        this.port = port;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets list of contact points to connect to Cassandra cluster.
 +     *
 +     * @param points contact points
 +     */
 +    public void setContactPoints(String... points) {
 +        if (points == null || points.length == 0)
 +            return;
 +
 +        for (String point : points) {
 +            if (point.contains(":")) {
 +                if (contactPointsWithPorts == null)
 +                    contactPointsWithPorts = new LinkedList<>();
 +
 +                String[] chunks = point.split(":");
 +
 +                try {
 +                    contactPointsWithPorts.add(InetSocketAddress.createUnresolved(chunks[0].trim(), Integer.parseInt(chunks[1].trim())));
 +                }
 +                catch (Throwable e) {
 +                    throw new IllegalArgumentException("Incorrect contact point '" + point + "' specified for Cassandra cache storage", e);
 +                }
 +            }
 +            else {
 +                if (contactPoints == null)
 +                    contactPoints = new LinkedList<>();
 +
 +                try {
 +                    contactPoints.add(InetAddress.getByName(point));
 +                }
 +                catch (Throwable e) {
 +                    throw new IllegalArgumentException("Incorrect contact point '" + point + "' specified for Cassandra cache storage", e);
 +                }
 +            }
 +        }
 +
 +        invalidate();
 +    }
 +
 +    /** Sets maximum time to wait for schema agreement before returning from a DDL query. */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setMaxSchemaAgreementWaitSeconds(int seconds) {
 +        maxSchemaAgreementWaitSeconds = seconds;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets the native protocol version to use.
 +     *
 +     * @param ver version number
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setProtocolVersion(int ver) {
 +        protoVer = ver;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets compression algorithm to use for the transport.
 +     *
 +     * @param compression Compression algorithm.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setCompression(String compression) {
 +        this.compression = compression == null || compression.trim().isEmpty() ? null : compression.trim();
 +
 +        try {
 +            if (this.compression != null)
 +                ProtocolOptions.Compression.valueOf(this.compression);
 +        }
 +        catch (Throwable e) {
 +            throw new IgniteException("Incorrect compression '" + compression + "' specified for Cassandra connection", e);
 +        }
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Enables SSL for communications with Cassandra.
 +     *
 +     * @param use Flag to enable/disable SSL.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setUseSSL(boolean use) {
 +        useSSL = use;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Enables metrics collection.
 +     *
 +     * @param collect Flag to enable/disable metrics collection.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setCollectMetrix(boolean collect) {
 +        collectMetrix = collect;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Enables JMX reporting of the metrics.
 +     *
 +     * @param enableReporting Flag to enable/disable JMX reporting.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setJmxReporting(boolean enableReporting) {
 +        jmxReporting = enableReporting;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets number of rows to immediately fetch in CQL statement execution.
 +     *
 +     * @param size Number of rows to fetch.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setFetchSize(int size) {
 +        fetchSize = size;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Set consistency level for READ operations.
 +     *
 +     * @param level Consistency level.
 +     */
 +    public void setReadConsistency(String level) {
 +        readConsistency = parseConsistencyLevel(level);
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Set consistency level for WRITE operations.
 +     *
 +     * @param level Consistency level.
 +     */
 +    public void setWriteConsistency(String level) {
 +        writeConsistency = parseConsistencyLevel(level);
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets credentials to use for authentication.
 +     *
 +     * @param creds Credentials.
 +     */
 +    public void setCredentials(Credentials creds) {
 +        this.creds = creds;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets load balancing policy.
 +     *
 +     * @param plc Load balancing policy.
 +     */
 +    public void setLoadBalancingPolicy(LoadBalancingPolicy plc) {
 +        loadBalancingPlc = plc;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets reconnection policy.
 +     *
 +     * @param plc Reconnection policy.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setReconnectionPolicy(ReconnectionPolicy plc) {
 +        reconnectionPlc = plc;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets retry policy.
 +     *
 +     * @param plc Retry policy.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setRetryPolicy(RetryPolicy plc) {
 +        retryPlc = plc;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets address translator.
 +     *
 +     * @param translator Address translator.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setAddressTranslator(AddressTranslator translator) {
 +        addrTranslator = translator;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets speculative execution policy.
 +     *
 +     * @param plc Speculative execution policy.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setSpeculativeExecutionPolicy(SpeculativeExecutionPolicy plc) {
 +        speculativeExecutionPlc = plc;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets authentication provider.
 +     *
 +     * @param provider Authentication provider.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setAuthProvider(AuthProvider provider) {
 +        authProvider = provider;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets SSL options.
 +     *
 +     * @param options SSL options.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setSslOptions(SSLOptions options) {
 +        sslOptions = options;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets pooling options.
 +     *
 +     * @param options pooling options to use.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setPoolingOptions(PoolingOptions options) {
 +        poolingOptions = options;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets socket options to use.
 +     *
 +     * @param options Socket options.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setSocketOptions(SocketOptions options) {
 +        sockOptions = options;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets netty options to use.
 +     *
 +     * @param options netty options.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setNettyOptions(NettyOptions options) {
 +        nettyOptions = options;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Creates Cassandra session wrapper if it wasn't created yet and returns it
 +     *
 +     * @param log logger
 +     * @return Cassandra session wrapper
 +     */
 +    @SuppressWarnings("deprecation")
 +    public synchronized CassandraSession session(IgniteLogger log) {
 +        if (ses != null)
 +            return ses;
 +
 +        Cluster.Builder builder = Cluster.builder();
 +
 +        if (user != null)
 +            builder = builder.withCredentials(user, pwd);
 +
 +        if (port != null)
 +            builder = builder.withPort(port);
 +
 +        if (contactPoints != null)
 +            builder = builder.addContactPoints(contactPoints);
 +
 +        if (contactPointsWithPorts != null)
 +            builder = builder.addContactPointsWithPorts(contactPointsWithPorts);
 +
 +        if (maxSchemaAgreementWaitSeconds != null)
 +            builder = builder.withMaxSchemaAgreementWaitSeconds(maxSchemaAgreementWaitSeconds);
 +
 +        if (protoVer != null)
 +            builder = builder.withProtocolVersion(ProtocolVersion.fromInt(protoVer));
 +
 +        if (compression != null) {
 +            try {
 +                builder = builder.withCompression(ProtocolOptions.Compression.valueOf(compression.trim().toLowerCase()));
 +            }
 +            catch (IllegalArgumentException e) {
 +                throw new IgniteException("Incorrect compression option '" + compression + "' specified for Cassandra connection", e);
 +            }
 +        }
 +
 +        if (useSSL != null && useSSL)
 +            builder = builder.withSSL();
 +
 +        if (sslOptions != null)
 +            builder = builder.withSSL(sslOptions);
 +
 +        if (collectMetrix != null && !collectMetrix)
 +            builder = builder.withoutMetrics();
 +
 +        if (jmxReporting != null && !jmxReporting)
 +            builder = builder.withoutJMXReporting();
 +
 +        if (creds != null)
 +            builder = builder.withCredentials(creds.getUser(), creds.getPassword());
 +
 +        if (loadBalancingPlc != null)
 +            builder = builder.withLoadBalancingPolicy(loadBalancingPlc);
 +
 +        if (reconnectionPlc != null)
 +            builder = builder.withReconnectionPolicy(reconnectionPlc);
 +
 +        if (retryPlc != null)
 +            builder = builder.withRetryPolicy(retryPlc);
 +
 +        if (addrTranslator != null)
 +            builder = builder.withAddressTranslator(addrTranslator);
 +
 +        if (speculativeExecutionPlc != null)
 +            builder = builder.withSpeculativeExecutionPolicy(speculativeExecutionPlc);
 +
 +        if (authProvider != null)
 +            builder = builder.withAuthProvider(authProvider);
 +
 +        if (poolingOptions != null)
 +            builder = builder.withPoolingOptions(poolingOptions);
 +
 +        if (sockOptions != null)
 +            builder = builder.withSocketOptions(sockOptions);
 +
 +        if (nettyOptions != null)
 +            builder = builder.withNettyOptions(nettyOptions);
 +
 +        return ses = new CassandraSessionImpl(builder, fetchSize, readConsistency, writeConsistency, log);
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public void writeExternal(ObjectOutput out) throws IOException {
 +        out.writeObject(fetchSize);
 +        out.writeObject(readConsistency);
 +        out.writeObject(writeConsistency);
 +        U.writeString(out, user);
 +        U.writeString(out, pwd);
 +        out.writeObject(port);
 +        out.writeObject(contactPoints);
 +        out.writeObject(contactPointsWithPorts);
 +        out.writeObject(maxSchemaAgreementWaitSeconds);
 +        out.writeObject(protoVer);
 +        U.writeString(out, compression);
 +        out.writeObject(useSSL);
 +        out.writeObject(collectMetrix);
 +        out.writeObject(jmxReporting);
 +        out.writeObject(creds);
 +        writeObject(out, loadBalancingPlc);
 +        writeObject(out, reconnectionPlc);
 +        writeObject(out, addrTranslator);
 +        writeObject(out, speculativeExecutionPlc);
 +        writeObject(out, authProvider);
 +        writeObject(out, sslOptions);
 +        writeObject(out, poolingOptions);
 +        writeObject(out, sockOptions);
 +        writeObject(out, nettyOptions);
 +    }
 +
 +    /** {@inheritDoc} */
 +    @SuppressWarnings("unchecked")
 +    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
 +        fetchSize = (Integer)in.readObject();
 +        readConsistency = (ConsistencyLevel)in.readObject();
 +        writeConsistency = (ConsistencyLevel)in.readObject();
 +        user = U.readString(in);
 +        pwd = U.readString(in);
 +        port = (Integer)in.readObject();
 +        contactPoints = (List<InetAddress>)in.readObject();
 +        contactPointsWithPorts = (List<InetSocketAddress>)in.readObject();
 +        maxSchemaAgreementWaitSeconds = (Integer)in.readObject();
 +        protoVer = (Integer)in.readObject();
 +        compression = U.readString(in);
 +        useSSL = (Boolean)in.readObject();
 +        collectMetrix = (Boolean)in.readObject();
 +        jmxReporting = (Boolean)in.readObject();
 +        creds = (Credentials)in.readObject();
 +        loadBalancingPlc = (LoadBalancingPolicy)readObject(in);
 +        reconnectionPlc = (ReconnectionPolicy)readObject(in);
 +        addrTranslator = (AddressTranslator)readObject(in);
 +        speculativeExecutionPlc = (SpeculativeExecutionPolicy)readObject(in);
 +        authProvider = (AuthProvider)readObject(in);
 +        sslOptions = (SSLOptions)readObject(in);
 +        poolingOptions = (PoolingOptions)readObject(in);
 +        sockOptions = (SocketOptions)readObject(in);
 +        nettyOptions = (NettyOptions)readObject(in);
 +    }
 +
 +    /**
 +     * Helper method used to serialize class members
 +     * @param out the stream to write the object to
 +     * @param obj the object to be written
 +     * @throws IOException Includes any I/O exceptions that may occur
 +     */
 +    private void writeObject(ObjectOutput out, Object obj) throws IOException {
 +        out.writeObject(obj == null || !(obj instanceof Serializable) ? NULL_OBJECT : obj);
 +    }
 +
 +    /**
 +     * Helper method used to deserialize class members
 +     * @param in the stream to read data from in order to restore the object
 +     * @throws IOException Includes any I/O exceptions that may occur
 +     * @throws ClassNotFoundException If the class for an object being restored cannot be found
 +     * @return deserialized object
 +     */
 +    private Object readObject(ObjectInput in) throws IOException, ClassNotFoundException {
 +        Object obj = in.readObject();
 +        return NULL_OBJECT.equals(obj) ? null : obj;
 +    }
 +
 +    /**
 +     * Parses consistency level provided as string.
 +     *
 +     * @param level consistency level string.
 +     *
 +     * @return consistency level.
 +     */
 +    private ConsistencyLevel parseConsistencyLevel(String level) {
 +        if (level == null)
 +            return null;
 +
 +        try {
 +            return ConsistencyLevel.valueOf(level.trim().toUpperCase());
 +        }
 +        catch (Throwable e) {
 +            throw new IgniteException("Incorrect consistency level '" + level + "' specified for Cassandra connection", e);
 +        }
 +    }
 +
 +    /**
 +     * Invalidates session.
 +     */
 +    private synchronized void invalidate() {
 +        ses = null;
 +    }
++
++    /** {@inheritDoc} */
++    @Override public String toString() {
++        return S.toString(DataSource.class, this);
++    }
 +}

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/IgniteServices.java
----------------------------------------------------------------------

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

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java
----------------------------------------------------------------------

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 5f0b8a0,4d59d50..8187e8f
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@@ -3076,8 -3233,8 +3076,8 @@@ public abstract class GridCacheAdapter<
      }
  
      /** {@inheritDoc} */
 -    @Override public CacheMetrics clusterMetrics() {
 +    @Override public final CacheMetrics clusterMetrics() {
-         return clusterMetrics(ctx.grid().cluster().forCacheNodes(ctx.name()));
+         return clusterMetrics(ctx.grid().cluster().forDataNodes(ctx.name()));
      }
  
      /** {@inheritDoc} */

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index aeb3ef4,3690f35..d26242d
--- 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
@@@ -1587,9 -1590,9 +1590,9 @@@ public class GridServiceProcessor exten
                  else
                      topVer = new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion(), 0);
  
 -                depExe.submit(new BusyRunnable() {
 +                depExe.execute(new BusyRunnable() {
                      @Override public void run0() {
-                         ClusterNode oldest = CU.oldestAliveCacheServerNode(cache.context().shared(), topVer);
+                         ClusterNode oldest = ctx.discovery().oldestAliveCacheServerNode(topVer);
  
                          if (oldest != null && oldest.isLocal()) {
                              final Collection<GridServiceDeployment> retries = new ConcurrentLinkedQueue<>();

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
index ee5b65c,43017db..8f8d78a
--- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
@@@ -104,24 -104,9 +104,25 @@@ public abstract class AbstractAffinityF
      }
  
      /**
+      * @param backups Number of backups.
       * @throws Exception If failed.
       */
 +    public void testNullKeyForPartitionCalculation() throws Exception {
 +        AffinityFunction aff = affinityFunction();
 +
 +        try {
 +            aff.partition(null);
 +
 +            fail("Should throw IllegalArgumentException due to NULL affinity key.");
 +        } catch (IllegalArgumentException e) {
 +            e.getMessage().contains("Null key is passed for a partition calculation. " +
 +                "Make sure that an affinity key that is used is initialized properly.");
 +        }
 +    }
 +
 +    /**
 +     * @throws Exception If failed.
 +     */
      protected void checkNodeRemoved(int backups) throws Exception {
          checkNodeRemoved(backups, 1, 1);
      }

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/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 554bb3d,deec72a..1e73e79
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@@ -39,10 -39,7 +39,11 @@@ import org.apache.ignite.cache.store.jd
  import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreTest;
  import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreMultithreadedSelfTest;
  import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreSelfTest;
 +import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceMultipleConnectionsTest;
 +import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceTest;
 +import org.apache.ignite.internal.managers.communication.IgniteIoTestMessagesTest;
 +import org.apache.ignite.internal.managers.communication.IgniteVariousConnectionNumberTest;
+ import org.apache.ignite.cache.store.jdbc.JdbcTypesDefaultTransformerTest;
  import org.apache.ignite.internal.processors.cache.CacheAffinityCallSelfTest;
  import org.apache.ignite.internal.processors.cache.CacheDeferredDeleteSanitySelfTest;
  import org.apache.ignite.internal.processors.cache.CacheEntryProcessorCopySelfTest;

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/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 2d06f3a,350b715..b28619c
--- 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,7 -66,7 +66,8 @@@ 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.IgniteServiceProxyTimeoutInitializedTest;
  import org.apache.ignite.internal.processors.service.IgniteServiceReassignmentTest;
  import org.apache.ignite.internal.processors.service.ServicePredicateAccessCacheTest;
  import org.apache.ignite.internal.util.GridStartupWithUndefinedIgniteHomeSelfTest;
@@@ -142,7 -142,7 +143,8 @@@ public class IgniteKernalSelfTestSuite 
          suite.addTestSuite(GridServiceProxyNodeStopSelfTest.class);
          suite.addTestSuite(GridServiceProxyClientReconnectSelfTest.class);
          suite.addTestSuite(IgniteServiceReassignmentTest.class);
+         suite.addTestSuite(IgniteServiceProxyTimeoutInitializedTest.class);
 +        suite.addTestSuite(IgniteServiceDynamicCachesSelfTest.class);
  
          suite.addTestSuite(IgniteServiceDeploymentClassLoadingDefaultMarshallerTest.class);
          suite.addTestSuite(IgniteServiceDeploymentClassLoadingOptimizedMarshallerTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/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 5df44db,362ddd8..c541185
--- 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
@@@ -794,40 -774,29 +794,45 @@@ public class IgniteH2Indexing implement
          throws IgniteCheckedException {
          final Connection conn = connectionForSpace(spaceName);
  
-         initLocalQueryContext(conn, enforceJoinOrder, filters);
+         setupConnection(conn, false, enforceJoinOrder);
  
-         Prepared p = null;
+         final PreparedStatement stmt = preparedStatementWithParams(conn, qry, params, true);
  
-         try {
-             final PreparedStatement stmt = preparedStatementWithParams(conn, qry, params, true);
++        Prepared p = GridSqlQueryParser.prepared((JdbcPreparedStatement)stmt);
 +
-             p = GridSqlQueryParser.prepared((JdbcPreparedStatement) stmt);
++        if (!p.isQuery()) {
++            GridH2QueryContext.clearThreadLocal();
 +
-             if (!p.isQuery()) {
-                 GridH2QueryContext.clearThreadLocal();
++            SqlFieldsQuery fldsQry = new SqlFieldsQuery(qry);
 +
-                 SqlFieldsQuery fldsQry = new SqlFieldsQuery(qry);
++            if (params != null)
++                fldsQry.setArgs(params.toArray());
 +
-                 if (params != null)
-                     fldsQry.setArgs(params.toArray());
++            fldsQry.setEnforceJoinOrder(enforceJoinOrder);
++            fldsQry.setTimeout(timeout, TimeUnit.MILLISECONDS);
 +
-                 fldsQry.setEnforceJoinOrder(enforceJoinOrder);
-                 fldsQry.setTimeout(timeout, TimeUnit.MILLISECONDS);
++            return dmlProc.updateLocalSqlFields(spaceName, stmt, fldsQry, filters, cancel);
++        }
 +
-                 return dmlProc.updateLocalSqlFields(spaceName, stmt, fldsQry, filters, cancel);
-             }
+         List<GridQueryFieldMetadata> meta;
  
-             List<GridQueryFieldMetadata> meta;
+         try {
+             meta = meta(stmt.getMetaData());
+         }
+         catch (SQLException e) {
+             throw new IgniteCheckedException("Cannot prepare query metadata", e);
+         }
  
-             try {
-                 meta = meta(stmt.getMetaData());
-             }
-             catch (SQLException e) {
-                 throw new IgniteCheckedException("Cannot prepare query metadata", e);
-             }
+         final GridH2QueryContext ctx = new GridH2QueryContext(nodeId, nodeId, 0, LOCAL)
+             .filter(filters).distributedJoins(false);
  
-             return new GridQueryFieldsResultAdapter(meta, null) {
-                 @Override public GridCloseableIterator<List<?>> iterator() throws IgniteCheckedException{
+         return new GridQueryFieldsResultAdapter(meta, null) {
+             @Override public GridCloseableIterator<List<?>> iterator() throws IgniteCheckedException {
+                 assert GridH2QueryContext.get() == null;
+ 
+                 GridH2QueryContext.set(ctx);
+ 
+                 try {
                      ResultSet rs = executeSqlQueryWithTimer(spaceName, stmt, conn, qry, params, timeout, cancel);
  
                      return new FieldsIterator(rs);

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


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

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

# Conflicts:
#	modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
#	modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java
#	modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
#	modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
#	modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
#	modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
#	modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 09c634e28200e0dd94c3c2e2271b5cdf60304888
Parents: 64247b9
Author: sboikov <sb...@gridgain.com>
Authored: Mon Dec 19 15:55:02 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Dec 19 15:55:02 2016 +0300

----------------------------------------------------------------------
 .../cache/store/cassandra/CassandraCacheStore.java  |  3 +++
 .../org/apache/ignite/internal/IgniteKernal.java    | 16 ----------------
 2 files changed, 3 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/09c634e2/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java b/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
index b4bed0d..70d798b 100644
--- a/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
+++ b/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
@@ -43,6 +43,9 @@ import org.apache.ignite.cache.store.cassandra.session.CassandraSession;
 import org.apache.ignite.cache.store.cassandra.session.ExecutionAssistant;
 import org.apache.ignite.cache.store.cassandra.session.GenericBatchExecutionAssistant;
 import org.apache.ignite.cache.store.cassandra.session.LoadCacheCustomQueryWorker;
+import org.apache.ignite.cache.store.cassandra.session.transaction.DeleteMutation;
+import org.apache.ignite.cache.store.cassandra.session.transaction.Mutation;
+import org.apache.ignite.cache.store.cassandra.session.transaction.WriteMutation;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiInClosure;

http://git-wip-us.apache.org/repos/asf/ignite/blob/09c634e2/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index e4a6115..4972d1f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -2877,22 +2877,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         }
     }
 
-    /**
-     * @param cacheNames Collection of cache names.
-     * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
-     * @return Ignite future.
-     */
-    public IgniteInternalFuture<?> destroyCachesAsync(Collection<String> cacheNames, boolean checkThreadTx) {
-        guard();
-
-        try {
-            return ctx.cache().dynamicDestroyCaches(cacheNames, checkThreadTx);
-        }
-        finally {
-            unguard();
-        }
-    }
-
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateCache(String cacheName) {
         guard();


[42/50] [abbrv] ignite git commit: IGNITE-4494 .NET: Optimize ExamplesTest.TestRemoteNodes

Posted by yz...@apache.org.
IGNITE-4494 .NET: Optimize ExamplesTest.TestRemoteNodes

Reuse standalone nodes between test runs


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 7606e6624be313a114a1c6350faa83a5f5063938
Parents: df725e8
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Dec 26 18:47:11 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Dec 26 18:47:11 2016 +0300

----------------------------------------------------------------------
 .../Examples/Example.cs                         |   5 +-
 .../Examples/ExamplesTest.cs                    | 107 +++++++++++++------
 2 files changed, 78 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7606e662/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/Example.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/Example.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/Example.cs
index 4df012c..e7a264d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/Example.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/Example.cs
@@ -23,7 +23,6 @@ namespace Apache.Ignite.Core.Tests.Examples
     using System.Linq;
     using System.Text.RegularExpressions;
     using Apache.Ignite.Examples.Compute;
-    using Apache.Ignite.ExamplesDll.Compute;
     using NUnit.Framework;
 
     /// <summary>
@@ -76,8 +75,6 @@ namespace Apache.Ignite.Core.Tests.Examples
 
             Assert.IsTrue(types.Any());
 
-            var examplesDllName = typeof(AverageSalaryJob).Assembly.GetName().Name;
-
             foreach (var type in types)
             {
                 var sourceFile = sourceFiles.Single(x => x.EndsWith(string.Format("\\{0}.cs", type.Name)));
@@ -87,7 +84,7 @@ namespace Apache.Ignite.Core.Tests.Examples
                 yield return new Example
                 {
                     ConfigPath = GetConfigPath(sourceCode),
-                    NeedsTestDll = sourceCode.Contains(examplesDllName),
+                    NeedsTestDll = sourceCode.Contains("-assembly="),
                     _runAction = GetRunAction(type),
                     Name = type.Name
                 };

http://git-wip-us.apache.org/repos/asf/ignite/blob/7606e662/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 61bfb5c..19e68a6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Examples/ExamplesTest.cs
@@ -32,8 +32,22 @@ namespace Apache.Ignite.Core.Tests.Examples
     public class ExamplesTest
     {
         /** */
+        private static readonly Example[] AllExamples = Example.GetExamples().ToArray();
+
+        /** */
+        private static readonly string[] LocalOnlyExamples =
+        {
+            "LifecycleExample", "ClientReconnectExample", "MultiTieredCacheExample"
+        };
+
+        /** */
+        private static readonly string[] NoDllExamples = { "BinaryModeExample", "NearCacheExample" };
+
+        /** */
         private IDisposable _changedConfig;
 
+        /** */
+        private bool _remoteNodeStarted;
         /// <summary>
         /// Tests the example in a single node mode.
         /// </summary>
@@ -41,6 +55,14 @@ namespace Apache.Ignite.Core.Tests.Examples
         [Test, TestCaseSource("TestCasesLocal")]
         public void TestLocalNode(Example example)
         {
+            StopRemoteNodes();
+
+            if (LocalOnlyExamples.Contains(example.Name))
+            {
+                Assert.IsFalse(example.NeedsTestDll, "Local-only example should not mention test dll.");
+                Assert.IsNull(example.ConfigPath, "Local-only example should not mention app.config path.");
+            }
+
             example.Run();
         }
 
@@ -69,41 +91,65 @@ namespace Apache.Ignite.Core.Tests.Examples
         /// </summary>
         /// <param name="example">The example to run.</param>
         /// <param name="clientMode">Client mode flag.</param>
-        private static void TestRemoteNodes(Example example, bool clientMode)
+        private void TestRemoteNodes(Example example, bool clientMode)
         {
-            Assert.IsNotEmpty(example.ConfigPath);
+            Assert.IsTrue(PathUtil.ExamplesAppConfigPath.EndsWith(example.ConfigPath,
+                StringComparison.OrdinalIgnoreCase), "All examples should use the same app.config.");
 
-            var configPath = Path.Combine(PathUtil.IgniteHome, PathUtil.DevPrefix, example.ConfigPath);
+            Assert.IsTrue(example.NeedsTestDll || NoDllExamples.Contains(example.Name),
+                "Examples that allow standalone nodes should mention test dll.");
 
-            // Try with multiple standalone nodes
-            for (var i = 0; i < 2; i++)
-            {
-                // Start a grid to monitor topology
-                // Stop it after topology check so we don't interfere with example
-                Ignition.ClientMode = false;
+            StartRemoteNodes();
 
-                using (var ignite = Ignition.StartFromApplicationConfiguration(
-                    "igniteConfiguration", configPath))
-                {
-                    var args = new List<string> { "-configFileName=" + configPath};
+            Ignition.ClientMode = clientMode;
 
-                    if (example.NeedsTestDll)
-                        args.Add(" -assembly=" + typeof(AverageSalaryJob).Assembly.Location);
+            // Run twice to catch issues with standalone node state
+            example.Run();
+            example.Run();
+        }
 
-                    var proc = new IgniteProcess(args.ToArray());
+        /// <summary>
+        /// Starts standalone node.
+        /// </summary>
+        private void StartRemoteNodes()
+        {
+            if (_remoteNodeStarted)
+                return;
+
+            // Start a grid to monitor topology;
+            // Stop it after topology check so we don't interfere with example.
+            Ignition.ClientMode = false;
 
-                    Assert.IsTrue(ignite.WaitTopology(i + 2), 
-                        string.Format("Standalone node failed to join topology: [{0}]", proc.GetInfo()));
+            using (var ignite = Ignition.StartFromApplicationConfiguration(
+                "igniteConfiguration", PathUtil.ExamplesAppConfigPath))
+            {
+                var args = new List<string>
+                {
+                    "-configFileName=" + PathUtil.ExamplesAppConfigPath,
+                    " -assembly=" + typeof(AverageSalaryJob).Assembly.Location
+                };
 
-                    Assert.IsTrue(proc.Alive, string.Format("Standalone node stopped unexpectedly: [{0}]", 
-                        proc.GetInfo()));
-                }
+                var proc = new IgniteProcess(args.ToArray());
 
-                Ignition.ClientMode = clientMode;
+                Assert.IsTrue(ignite.WaitTopology(2), 
+                    string.Format("Standalone node failed to join topology: [{0}]", proc.GetInfo()));
+
+                Assert.IsTrue(proc.Alive, string.Format("Standalone node stopped unexpectedly: [{0}]",
+                    proc.GetInfo()));
+            }
+
+            _remoteNodeStarted = true;
+        }
 
-                // Run twice to catch issues with standalone node state
-                example.Run();
-                example.Run();
+        /// <summary>
+        /// Stops standalone nodes.
+        /// </summary>
+        private void StopRemoteNodes()
+        {
+            if (_remoteNodeStarted)
+            {
+                IgniteProcess.KillAll();
+                _remoteNodeStarted = false;
             }
         }
 
@@ -127,6 +173,10 @@ namespace Apache.Ignite.Core.Tests.Examples
         public void FixtureTearDown()
         {
             _changedConfig.Dispose();
+
+            Ignition.StopAll(true);
+
+            IgniteProcess.KillAll();
         }
 
         /// <summary>
@@ -136,7 +186,6 @@ namespace Apache.Ignite.Core.Tests.Examples
         public void TearDown()
         {
             Ignition.ClientMode = false;
-            IgniteProcess.KillAll();
         }
 
         /// <summary>
@@ -146,7 +195,7 @@ namespace Apache.Ignite.Core.Tests.Examples
         // ReSharper disable once MemberCanBeMadeStatic.Global
         public IEnumerable<Example> TestCasesLocal
         {
-            get { return Example.GetExamples().Where(x => x.Name != "NearCacheExample"); }
+            get { return AllExamples.Where(x => x.Name != "NearCacheExample"); }
         }
 
         /// <summary>
@@ -158,9 +207,7 @@ namespace Apache.Ignite.Core.Tests.Examples
         {
             get
             {
-                var localOnly = new[] {"LifecycleExample", "ClientReconnectExample", "MultiTieredCacheExample" };
-
-                return Example.GetExamples().Where(x => !localOnly.Contains(x.Name));
+                return AllExamples.Where(x => !LocalOnlyExamples.Contains(x.Name));
             }
         }
     }


[36/50] [abbrv] ignite git commit: IGNITE-4471: Fixed ODBC string-reading utility function

Posted by yz...@apache.org.
IGNITE-4471: Fixed ODBC string-reading utility function

 This closes #1377


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 8cffe90010e1f0dbb0b93c33fe6a08bd27fa9477
Parents: dbc8a0f
Author: Igor Sapego <is...@gridgain.com>
Authored: Fri Dec 23 13:51:39 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Dec 23 13:51:39 2016 +0300

----------------------------------------------------------------------
 .../processors/odbc/OdbcRequestHandler.java     |   2 +-
 .../internal/processors/odbc/OdbcTableMeta.java |  15 +-
 .../cpp/odbc-test/config/queries-default.xml    | 145 +++++++++++++++++++
 .../odbc-test/config/queries-test-noodbc.xml    |  84 +----------
 .../cpp/odbc-test/config/queries-test.xml       | 122 +---------------
 .../cpp/odbc-test/src/queries_test.cpp          |  76 ++++++++++
 .../cpp/odbc-test/src/utility_test.cpp          |  27 +++-
 modules/platforms/cpp/odbc/src/utility.cpp      |  10 ++
 8 files changed, 271 insertions(+), 210 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8cffe900/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
index 2e0fd10..f922d9a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
@@ -388,7 +388,7 @@ public class OdbcRequestHandler {
                     if (!matches("TABLE", req.tableType()))
                         continue;
 
-                    OdbcTableMeta tableMeta = new OdbcTableMeta(req.catalog(), cacheName, table.name(), "TABLE");
+                    OdbcTableMeta tableMeta = new OdbcTableMeta(null, cacheName, table.name(), "TABLE");
 
                     if (!meta.contains(tableMeta))
                         meta.add(tableMeta);

http://git-wip-us.apache.org/repos/asf/ignite/blob/8cffe900/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTableMeta.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTableMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTableMeta.java
index fb7df50..ca630dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTableMeta.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTableMeta.java
@@ -19,6 +19,8 @@ package org.apache.ignite.internal.processors.odbc;
 
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
 
+import java.util.Objects;
+
 /**
  * ODBC table-related metadata.
  */
@@ -50,11 +52,11 @@ public class OdbcTableMeta {
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        int hash = catalog.hashCode();
+        int hash = Objects.hashCode(catalog);
 
-        hash = 31 * hash + schema.hashCode();
-        hash = 31 * hash + table.hashCode();
-        hash = 31 * hash + tableType.hashCode();
+        hash = 31 * hash + Objects.hashCode(schema);
+        hash = 31 * hash + Objects.hashCode(table);
+        hash = 31 * hash + Objects.hashCode(tableType);
 
         return hash;
     }
@@ -64,8 +66,9 @@ public class OdbcTableMeta {
         if (o instanceof OdbcTableMeta) {
             OdbcTableMeta other = (OdbcTableMeta) o;
 
-            return this == other || catalog.equals(other.catalog) && schema.equals(other.schema) &&
-                table.equals(other.table) && tableType.equals(other.tableType);
+            return this == other ||
+                    Objects.equals(catalog, other.catalog) && Objects.equals(schema, other.schema) &&
+                    Objects.equals(table, other.table) && Objects.equals(tableType, other.tableType);
         }
 
         return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8cffe900/modules/platforms/cpp/odbc-test/config/queries-default.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/config/queries-default.xml b/modules/platforms/cpp/odbc-test/config/queries-default.xml
new file mode 100644
index 0000000..6614e93
--- /dev/null
+++ b/modules/platforms/cpp/odbc-test/config/queries-default.xml
@@ -0,0 +1,145 @@
+<?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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+
+    <bean abstract="true" id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+        <!--<property name="odbcConfiguration"><null/></property>-->
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+
+                    <!-- Configure type metadata to enable queries. -->
+                    <property name="queryEntities">
+                        <list>
+                            <bean class="org.apache.ignite.cache.QueryEntity">
+                                <property name="keyType" value="java.lang.Long"/>
+                                <property name="valueType" value="TestType"/>
+
+                                <property name="fields">
+                                    <map>
+                                        <entry key="i8Field" value="java.lang.Byte"/>
+                                        <entry key="i16Field" value="java.lang.Short"/>
+                                        <entry key="i32Field" value="java.lang.Integer"/>
+                                        <entry key="i64Field" value="java.lang.Long"/>
+                                        <entry key="strField" value="java.lang.String"/>
+                                        <entry key="floatField" value="java.lang.Float"/>
+                                        <entry key="doubleField" value="java.lang.Double"/>
+                                        <entry key="boolField" value="java.lang.Boolean"/>
+                                        <entry key="guidField" value="java.util.UUID"/>
+                                        <entry key="dateField" value="java.util.Date"/>
+                                        <entry key="timestampField" value="java.sql.Timestamp"/>
+                                    </map>
+                                </property>
+
+                                <property name="keyFields">
+                                    <list></list>
+                                </property>
+
+                                <property name="indexes">
+                                    <list>
+                                        <bean class="org.apache.ignite.cache.QueryIndex">
+                                            <constructor-arg value="i32Field"/>
+                                        </bean>
+                                        <bean class="org.apache.ignite.cache.QueryIndex">
+                                            <constructor-arg value="i64Field"/>
+                                        </bean>
+                                    </list>
+                                </property>
+                            </bean>
+                        </list>
+                    </property>
+                </bean>
+
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache2"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+
+                    <!-- Configure type metadata to enable queries. -->
+                    <property name="queryEntities">
+                        <list>
+                            <bean class="org.apache.ignite.cache.QueryEntity">
+                                <property name="keyType" value="java.lang.Long"/>
+                                <property name="valueType" value="ComplexType"/>
+
+                                <property name="fields">
+                                    <map>
+                                        <entry key="i32Field" value="java.lang.Integer"/>
+                                        <entry key="objField" value="TestObject"/>
+                                        <entry key="strField" value="java.lang.String"/>
+                                    </map>
+                                </property>
+
+                                <property name="keyFields">
+                                    <list></list>
+                                </property>
+
+                                <property name="indexes">
+                                    <list>
+                                        <bean class="org.apache.ignite.cache.QueryIndex">
+                                            <constructor-arg value="i32Field"/>
+                                        </bean>
+                                    </list>
+                                </property>
+                            </bean>
+                        </list>
+                    </property>
+                </bean>
+            </list>
+        </property>
+
+        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <!--
+                        Ignite provides several options for automatic discovery that can be used
+                        instead os static IP based discovery.
+                    -->
+                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+                <property name="socketTimeout" value="300" />
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/8cffe900/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml b/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml
index 82173de..b21287f 100644
--- a/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml
+++ b/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml
@@ -22,86 +22,10 @@
        xmlns:util="http://www.springframework.org/schema/util"
        xsi:schemaLocation="
         http://www.springframework.org/schema/beans
-        http://www.springframework.org/schema/beans/spring-beans.xsd
-        http://www.springframework.org/schema/util
-        http://www.springframework.org/schema/util/spring-util.xsd">
-    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
-        <property name="localHost" value="127.0.0.1"/>
-        <property name="connectorConfiguration"><null/></property>
-        <property name="odbcConfiguration"><null/></property>
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
 
-        <property name="cacheConfiguration">
-            <list>
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="cache"/>
-                    <property name="cacheMode" value="PARTITIONED"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+    <!-- Imports no-ODBC Ignite configuration -->
+    <import resource="queries-default.xml"/>
 
-                    <!-- Configure type metadata to enable queries. -->
-                    <property name="queryEntities">
-                        <list>
-                            <bean class="org.apache.ignite.cache.QueryEntity">
-                                <property name="keyType" value="java.lang.Long"/>
-                                <property name="valueType" value="TestType"/>
-
-                                <property name="fields">
-                                    <map>
-                                        <entry key="i8Field" value="java.lang.Byte"/>
-                                        <entry key="i16Field" value="java.lang.Short"/>
-                                        <entry key="i32Field" value="java.lang.Integer"/>
-                                        <entry key="i64Field" value="java.lang.Long"/>
-                                        <entry key="strField" value="java.lang.String"/>
-                                        <entry key="floatField" value="java.lang.Float"/>
-                                        <entry key="doubleField" value="java.lang.Double"/>
-                                        <entry key="boolField" value="java.lang.Boolean"/>
-                                        <entry key="guidField" value="java.util.UUID"/>
-                                        <entry key="dateField" value="java.util.Date"/>
-                                        <entry key="timestampField" value="java.sql.Timestamp"/>
-                                    </map>
-                                </property>
-
-                                <property name="keyFields">
-                                    <list></list>
-                                </property>
-
-                                <property name="indexes">
-                                    <list>
-                                        <bean class="org.apache.ignite.cache.QueryIndex">
-                                            <constructor-arg value="i32Field"/>
-                                        </bean>
-                                        <bean class="org.apache.ignite.cache.QueryIndex">
-                                            <constructor-arg value="i64Field"/>
-                                        </bean>
-                                    </list>
-                                </property>
-                            </bean>
-                        </list>
-                    </property>
-                </bean>
-            </list>
-        </property>
-
-        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
-        <property name="discoverySpi">
-            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
-                <property name="ipFinder">
-                    <!--
-                        Ignite provides several options for automatic discovery that can be used
-                        instead os static IP based discovery.
-                    -->
-                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
-                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
-                        <property name="addresses">
-                            <list>
-                                <!-- In distributed environment, replace with actual host IP address. -->
-                                <value>127.0.0.1:47500</value>
-                            </list>
-                        </property>
-                    </bean>
-                </property>
-                <property name="socketTimeout" value="300" />
-            </bean>
-        </property>
-    </bean>
+    <bean parent="ignite.cfg"/>
 </beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/8cffe900/modules/platforms/cpp/odbc-test/config/queries-test.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/config/queries-test.xml b/modules/platforms/cpp/odbc-test/config/queries-test.xml
index 906fadf..d08d4f1 100644
--- a/modules/platforms/cpp/odbc-test/config/queries-test.xml
+++ b/modules/platforms/cpp/odbc-test/config/queries-test.xml
@@ -22,129 +22,17 @@
        xmlns:util="http://www.springframework.org/schema/util"
        xsi:schemaLocation="
         http://www.springframework.org/schema/beans
-        http://www.springframework.org/schema/beans/spring-beans.xsd
-        http://www.springframework.org/schema/util
-        http://www.springframework.org/schema/util/spring-util.xsd">
-    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
-        <property name="localHost" value="127.0.0.1"/>
-        <property name="connectorConfiguration"><null/></property>
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
 
+    <!-- Imports no-ODBC Ignite configuration -->
+    <import resource="queries-default.xml"/>
+
+    <bean parent="ignite.cfg">
         <!-- Enabling ODBC. -->
         <property name="odbcConfiguration">
             <bean class="org.apache.ignite.configuration.OdbcConfiguration">
                 <property name="endpointAddress" value="127.0.0.1:11110"/>
             </bean>
         </property>
-
-        <property name="cacheConfiguration">
-            <list>
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="cache"/>
-                    <property name="cacheMode" value="PARTITIONED"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
-
-                    <!-- Configure type metadata to enable queries. -->
-                    <property name="queryEntities">
-                        <list>
-                            <bean class="org.apache.ignite.cache.QueryEntity">
-                                <property name="keyType" value="java.lang.Long"/>
-                                <property name="valueType" value="TestType"/>
-
-                                <property name="fields">
-                                    <map>
-                                        <entry key="i8Field" value="java.lang.Byte"/>
-                                        <entry key="i16Field" value="java.lang.Short"/>
-                                        <entry key="i32Field" value="java.lang.Integer"/>
-                                        <entry key="i64Field" value="java.lang.Long"/>
-                                        <entry key="strField" value="java.lang.String"/>
-                                        <entry key="floatField" value="java.lang.Float"/>
-                                        <entry key="doubleField" value="java.lang.Double"/>
-                                        <entry key="boolField" value="java.lang.Boolean"/>
-                                        <entry key="guidField" value="java.util.UUID"/>
-                                        <entry key="dateField" value="java.util.Date"/>
-                                        <entry key="timestampField" value="java.sql.Timestamp"/>
-                                    </map>
-                                </property>
-
-                                <property name="keyFields">
-                                    <list></list>
-                                </property>
-
-                                <property name="indexes">
-                                    <list>
-                                        <bean class="org.apache.ignite.cache.QueryIndex">
-                                            <constructor-arg value="i32Field"/>
-                                        </bean>
-                                        <bean class="org.apache.ignite.cache.QueryIndex">
-                                            <constructor-arg value="i64Field"/>
-                                        </bean>
-                                    </list>
-                                </property>
-                            </bean>
-                        </list>
-                    </property>
-                </bean>
-
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="cache2"/>
-                    <property name="cacheMode" value="PARTITIONED"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
-
-                    <!-- Configure type metadata to enable queries. -->
-                    <property name="queryEntities">
-                        <list>
-                            <bean class="org.apache.ignite.cache.QueryEntity">
-                                <property name="keyType" value="java.lang.Long"/>
-                                <property name="valueType" value="ComplexType"/>
-
-                                <property name="fields">
-                                    <map>
-                                        <entry key="i32Field" value="java.lang.Integer"/>
-                                        <entry key="objField" value="TestObject"/>
-                                        <entry key="strField" value="java.lang.String"/>
-                                    </map>
-                                </property>
-
-                                <property name="keyFields">
-                                    <list></list>
-                                </property>
-
-                                <property name="indexes">
-                                    <list>
-                                        <bean class="org.apache.ignite.cache.QueryIndex">
-                                            <constructor-arg value="i32Field"/>
-                                        </bean>
-                                    </list>
-                                </property>
-                            </bean>
-                        </list>
-                    </property>
-                </bean>
-            </list>
-        </property>
-
-        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
-        <property name="discoverySpi">
-            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
-                <property name="ipFinder">
-                    <!--
-                        Ignite provides several options for automatic discovery that can be used
-                        instead os static IP based discovery.
-                    -->
-                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
-                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
-                        <property name="addresses">
-                            <list>
-                                <!-- In distributed environment, replace with actual host IP address. -->
-                                <value>127.0.0.1:47500</value>
-                            </list>
-                        </property>
-                    </bean>
-                </property>
-                <property name="socketTimeout" value="300" />
-            </bean>
-        </property>
     </bean>
 </beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/8cffe900/modules/platforms/cpp/odbc-test/src/queries_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp
index 52b885d..a304229 100644
--- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp
@@ -1342,4 +1342,80 @@ BOOST_AUTO_TEST_CASE(TestInsertMergeSelect)
     BOOST_CHECK_EQUAL(recordsNum, selectedRecordsNum);
 }
 
+template<size_t n, size_t k>
+void CheckMeta(char columns[n][k], SQLLEN columnsLen[n])
+{
+    std::string catalog(columns[0], columnsLen[0]);
+    std::string schema(columns[1], columnsLen[1]);
+    std::string table(columns[2], columnsLen[2]);
+    std::string tableType(columns[3], columnsLen[3]);
+
+    BOOST_CHECK_EQUAL(catalog, std::string(""));
+    BOOST_CHECK_EQUAL(tableType, std::string("TABLE"));
+    BOOST_CHECK_EQUAL(columnsLen[4], SQL_NULL_DATA);
+
+    if (schema == "\"cache\"")
+    {
+        BOOST_CHECK_EQUAL(table, std::string("TestType"));
+    }
+    else if (schema == "\"cache2\"")
+    {
+        BOOST_CHECK_EQUAL(table, std::string("ComplexType"));
+    }
+    else
+    {
+        BOOST_FAIL("Unknown schema: " + schema);
+    }
+}
+
+BOOST_AUTO_TEST_CASE(TestTablesMeta)
+{
+    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache2");
+
+    SQLRETURN ret;
+
+    enum { COLUMNS_NUM = 5 };
+
+    // Five collumns: TABLE_CAT, TABLE_SCHEM, TABLE_NAME, TABLE_TYPE, REMARKS
+    char columns[COLUMNS_NUM][ODBC_BUFFER_SIZE];
+    SQLLEN columnsLen[COLUMNS_NUM];
+
+    // Binding columns.
+    for (size_t i = 0; i < COLUMNS_NUM; ++i)
+    {
+        columnsLen[i] = ODBC_BUFFER_SIZE;
+
+        ret = SQLBindCol(stmt, static_cast<SQLSMALLINT>(i + 1), SQL_C_CHAR, columns[i], columnsLen[i], &columnsLen[i]);
+
+        if (!SQL_SUCCEEDED(ret))
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+    }
+
+    SQLCHAR catalogPattern[] = "";
+    SQLCHAR schemaPattern[] = "";
+    SQLCHAR tablePattern[] = "";
+    SQLCHAR tableTypePattern[] = "";
+
+    ret = SQLTables(stmt, catalogPattern, SQL_NTS, schemaPattern,
+        SQL_NTS, tablePattern, SQL_NTS, tableTypePattern, SQL_NTS);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    ret = SQLFetch(stmt);
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    CheckMeta<COLUMNS_NUM, ODBC_BUFFER_SIZE>(columns, columnsLen);
+
+    ret = SQLFetch(stmt);
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    CheckMeta<COLUMNS_NUM, ODBC_BUFFER_SIZE>(columns, columnsLen);
+
+    ret = SQLFetch(stmt);
+    BOOST_CHECK(ret == SQL_NO_DATA);
+}
+
 BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/8cffe900/modules/platforms/cpp/odbc-test/src/utility_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/utility_test.cpp b/modules/platforms/cpp/odbc-test/src/utility_test.cpp
index 6c4d104..a66860f 100644
--- a/modules/platforms/cpp/odbc-test/src/utility_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/utility_test.cpp
@@ -54,28 +54,43 @@ BOOST_AUTO_TEST_CASE(TestUtilityCopyStringToBuffer)
     BOOST_REQUIRE(!strcmp(buffer, str.substr(0, 10).c_str()));
 }
 
-BOOST_AUTO_TEST_CASE(TestUtilityReadString)
+BOOST_AUTO_TEST_CASE(TestUtilityWriteReadString)
 {
     using namespace ignite::impl::binary;
     using namespace ignite::impl::interop;
 
-    std::string inputStr("Hello World!");
-    std::string outputStr;
+    std::string inStr1("Hello World!");
+    std::string inStr2;
+    std::string inStr3("Lorem ipsum");
+
+    std::string outStr1;
+    std::string outStr2;
+    std::string outStr3;
+    std::string outStr4;
 
     ignite::impl::interop::InteropUnpooledMemory mem(1024);
     InteropOutputStream outStream(&mem);
     BinaryWriterImpl writer(&outStream, 0);
 
-    writer.WriteString(inputStr.data(), static_cast<int32_t>(inputStr.size()));
+    WriteString(writer, inStr1);
+    WriteString(writer, inStr2);
+    WriteString(writer, inStr3);
+    writer.WriteNull();
 
     outStream.Synchronize();
 
     InteropInputStream inStream(&mem);
     BinaryReaderImpl reader(&inStream);
 
-    ReadString(reader, outputStr);
+    ReadString(reader, outStr1);
+    ReadString(reader, outStr2);
+    ReadString(reader, outStr3);
+    ReadString(reader, outStr4);
 
-    BOOST_REQUIRE(inputStr == outputStr);
+    BOOST_REQUIRE(inStr1 == outStr1);
+    BOOST_REQUIRE(inStr2 == outStr2);
+    BOOST_REQUIRE(inStr3 == outStr3);
+    BOOST_REQUIRE(outStr4.empty());
 }
 
 BOOST_AUTO_TEST_SUITE_END()
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8cffe900/modules/platforms/cpp/odbc/src/utility.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/utility.cpp b/modules/platforms/cpp/odbc/src/utility.cpp
index 22191eb..756ab36 100644
--- a/modules/platforms/cpp/odbc/src/utility.cpp
+++ b/modules/platforms/cpp/odbc/src/utility.cpp
@@ -56,6 +56,7 @@ namespace ignite
         void ReadString(ignite::impl::binary::BinaryReaderImpl& reader, std::string& str)
         {
             int32_t strLen = reader.ReadString(0, 0);
+
             if (strLen > 0)
             {
                 str.resize(strLen);
@@ -63,7 +64,16 @@ namespace ignite
                 reader.ReadString(&str[0], static_cast<int32_t>(str.size()));
             }
             else
+            {
                 str.clear();
+
+                if (strLen == 0)
+                {
+                    char dummy;
+
+                    reader.ReadString(&dummy, sizeof(dummy));
+                }
+            }
         }
 
         void WriteString(ignite::impl::binary::BinaryWriterImpl& writer, const std::string & str)


[34/50] [abbrv] ignite git commit: Web console beta-7.

Posted by yz...@apache.org.
Web console beta-7.


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 8e7c852b5ff4a3837f2f33f3d9adc0c2318dbdf8
Parents: 1e2469f
Author: Andrey Novikov <an...@gridgain.com>
Authored: Fri Dec 23 16:34:10 2016 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Fri Dec 23 16:34:11 2016 +0700

----------------------------------------------------------------------
 modules/web-console/backend/app/agent.js        |   15 +
 modules/web-console/backend/app/browser.js      |   13 +
 modules/web-console/backend/app/mongo.js        |   24 +-
 modules/web-console/backend/routes/demo.js      |   17 +-
 modules/web-console/backend/routes/profile.js   |    3 +-
 .../web-console/backend/services/notebooks.js   |   14 +-
 .../web-console/backend/services/sessions.js    |    6 +-
 modules/web-console/backend/services/spaces.js  |   15 +
 modules/web-console/frontend/app/app.js         |    5 -
 .../controllers/reset-password.controller.js    |   14 +-
 .../frontend/app/data/event-groups.json         |  169 +
 .../frontend/app/data/event-types.json          |  169 -
 .../frontend/app/data/pom-dependencies.json     |   12 +-
 .../ui-ace-docker/ui-ace-docker.controller.js   |    2 +-
 .../directives/ui-ace-docker/ui-ace-docker.jade |    2 +-
 .../ui-ace-pojos/ui-ace-pojos.controller.js     |   12 +-
 .../ui-ace-pom/ui-ace-pom.controller.js         |    4 +-
 .../helpers/jade/form/form-field-dropdown.jade  |    5 +-
 .../helpers/jade/form/form-field-number.jade    |    3 +-
 .../app/helpers/jade/form/form-field-text.jade  |   19 +-
 .../frontend/app/helpers/jade/mixins.jade       |   52 +-
 .../frontend/app/modules/Demo/Demo.module.js    |    6 +-
 .../configuration/EventGroups.provider.js       |   30 -
 .../modules/configuration/Version.service.js    |    6 +-
 .../configuration/configuration.module.js       |   63 +-
 .../generator/AbstractTransformer.js            |   17 +
 .../modules/configuration/generator/Beans.js    |    5 +
 .../generator/ConfigurationGenerator.js         | 2795 +++++++-------
 .../configuration/generator/Custom.service.js   |   23 +
 .../configuration/generator/Docker.service.js   |    4 +-
 .../generator/JavaTransformer.service.js        | 2318 +++++------
 .../configuration/generator/Maven.service.js    |  234 ++
 .../configuration/generator/Pom.service.js      |  233 --
 .../generator/Properties.service.js             |    2 +-
 .../configuration/generator/Readme.service.js   |    2 +-
 .../generator/SharpTransformer.service.js       |  437 ++-
 .../generator/SpringTransformer.service.js      |  497 ++-
 .../defaults/Cache.platform.service.js          |   56 +
 .../generator/defaults/Cache.service.js         |  131 +
 .../defaults/Cluster.platform.service.js        |   43 +
 .../generator/defaults/Cluster.service.js       |  289 ++
 .../generator/defaults/Event-groups.service.js  |   27 +
 .../generator/defaults/IGFS.service.js          |   64 +
 .../defaults/cache.platform.provider.js         |   60 -
 .../generator/defaults/cache.provider.js        |  137 -
 .../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 ----------
 .../frontend/app/modules/sql/Notebook.data.js   |   11 +-
 .../app/modules/sql/Notebook.service.js         |    2 +-
 .../app/modules/sql/scan-filter-input.jade      |   39 -
 .../modules/sql/scan-filter-input.service.js    |   51 -
 .../frontend/app/modules/sql/sql.controller.js  |  211 +-
 .../frontend/app/modules/sql/sql.module.js      |    2 -
 .../app/modules/states/configuration.state.js   |    2 +
 .../configuration/caches/node-filter.jade       |    2 +-
 .../states/configuration/caches/query.jade      |    3 +
 .../states/configuration/caches/store.jade      |    4 +-
 .../configuration/clusters/checkpoint.jade      |   11 +-
 .../configuration/clusters/checkpoint/fs.jade   |    8 +-
 .../configuration/clusters/checkpoint/jdbc.jade |    8 +-
 .../configuration/clusters/checkpoint/s3.jade   |   25 +-
 .../clusters/collision/custom.jade              |    2 +-
 .../clusters/collision/job-stealing.jade        |    2 +-
 .../configuration/clusters/deployment.jade      |  129 +-
 .../states/configuration/clusters/events.jade   |    4 +-
 .../states/configuration/clusters/failover.jade |    4 +-
 .../clusters/general/discovery/zookeeper.jade   |    2 +-
 .../discovery/zookeeper/retrypolicy/custom.jade |    2 +-
 .../configuration/clusters/load-balancing.jade  |   23 +-
 .../configuration/clusters/logger/custom.jade   |    2 +-
 .../states/configuration/clusters/ssl.jade      |    2 +-
 .../summary/summary-zipper.service.js           |   37 +
 .../configuration/summary/summary.controller.js |  103 +-
 .../configuration/summary/summary.worker.js     |  123 +
 .../frontend/app/modules/user/Auth.service.js   |   11 +-
 .../frontend/app/services/JavaTypes.service.js  |   13 +-
 .../frontend/app/services/Messages.service.js   |   17 +-
 .../frontend/controllers/admin-controller.js    |  211 +-
 .../frontend/controllers/caches-controller.js   |   22 +-
 .../frontend/controllers/clusters-controller.js |   42 +-
 .../frontend/controllers/domains-controller.js  |   32 +-
 .../frontend/controllers/igfs-controller.js     |   20 +-
 .../frontend/controllers/profile-controller.js  |    3 +-
 .../gulpfile.babel.js/webpack/common.js         |   17 +-
 .../webpack/environments/development.js         |   14 +-
 .../webpack/environments/production.js          |    3 +-
 .../webpack/plugins/progress.js                 |   82 -
 modules/web-console/frontend/package.json       |  178 +-
 .../frontend/public/images/cache.png            |  Bin 23700 -> 24791 bytes
 .../frontend/public/images/domains.png          |  Bin 23828 -> 22131 bytes
 .../web-console/frontend/public/images/igfs.png |  Bin 14307 -> 14139 bytes
 .../frontend/public/images/query-chart.png      |  Bin 16637 -> 17142 bytes
 .../frontend/public/images/query-metadata.png   |  Bin 32298 -> 39361 bytes
 .../frontend/public/images/query-table.png      |  Bin 29189 -> 28065 bytes
 .../frontend/public/images/summary.png          |  Bin 31997 -> 33650 bytes
 .../stylesheets/_font-awesome-custom.scss       |   23 +-
 .../frontend/public/stylesheets/form-field.scss |   37 +
 .../frontend/public/stylesheets/style.scss      |  111 +-
 .../frontend/test/unit/JavaTypes.test.js        |   17 +-
 .../frontend/test/unit/Version.test.js          |    8 +-
 .../views/configuration/domains-import.jade     |    5 +-
 .../frontend/views/configuration/summary.jade   |   25 +-
 .../frontend/views/settings/admin.jade          |   85 +-
 .../frontend/views/sql/notebook-new.jade        |    2 +-
 modules/web-console/frontend/views/sql/sql.jade |  235 +-
 .../frontend/views/templates/alert.jade         |    2 +-
 .../frontend/views/templates/select.jade        |    2 +-
 112 files changed, 5577 insertions(+), 11296 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/backend/app/agent.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/app/agent.js b/modules/web-console/backend/app/agent.js
index f74a3f2..791ea50 100644
--- a/modules/web-console/backend/app/agent.js
+++ b/modules/web-console/backend/app/agent.js
@@ -314,6 +314,21 @@ module.exports.factory = function(_, fs, path, JSZip, socketio, settings, mongo)
 
         /**
          * @param {Boolean} demo Is need run command on demo node.
+         * @param {Array.<String>} nids Node ids.
+         * @returns {Promise}
+         */
+        queryResetDetailMetrics(demo, nids) {
+            const cmd = new Command(demo, 'exe')
+                .addParam('name', 'org.apache.ignite.internal.visor.compute.VisorGatewayTask')
+                .addParam('p1', nids)
+                .addParam('p2', 'org.apache.ignite.internal.visor.cache.VisorCacheResetQueryDetailMetricsTask')
+                .addParam('p3', 'java.lang.Void');
+
+            return this.executeRest(cmd);
+        }
+
+        /**
+         * @param {Boolean} demo Is need run command on demo node.
          * @param {String} cacheName Cache name.
          * @returns {Promise}
          */

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/backend/app/browser.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/app/browser.js b/modules/web-console/backend/app/browser.js
index 2710829..499d84d 100644
--- a/modules/web-console/backend/app/browser.js
+++ b/modules/web-console/backend/app/browser.js
@@ -162,6 +162,19 @@ module.exports.factory = (_, socketio, agentMgr, configure) => {
                         .catch((err) => cb(_errorToJson(err)));
                 });
 
+                // Collect cache query metrics and return result to browser.
+                socket.on('node:query:reset:metrics', (nids, cb) => {
+                    agentMgr.findAgent(accountId())
+                        .then((agent) => agent.queryResetDetailMetrics(demo, nids))
+                        .then((data) => {
+                            if (data.finished)
+                                return cb(null, data.result);
+
+                            cb(_errorToJson(data.error));
+                        })
+                        .catch((err) => cb(_errorToJson(err)));
+                });
+
                 // Return cache metadata from all nodes in grid.
                 socket.on('node:cache:metadata', (cacheName, cb) => {
                     agentMgr.findAgent(accountId())

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/backend/app/mongo.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/app/mongo.js b/modules/web-console/backend/app/mongo.js
index 0f38eb2..58ab119 100644
--- a/modules/web-console/backend/app/mongo.js
+++ b/modules/web-console/backend/app/mongo.js
@@ -247,6 +247,7 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose
         longQueryWarningTimeout: Number,
         sqlFunctionClasses: [String],
         snapshotableIndex: Boolean,
+        queryDetailMetricsSize: Number,
         statisticsEnabled: Boolean,
         managementEnabled: Boolean,
         readFromBackup: Boolean,
@@ -823,7 +824,24 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose
             Custom: {
                 className: String
             }
-        }]
+        }],
+        deploymentSpi: {
+            kind: {type: String, enum: ['URI', 'Local', 'Custom']},
+            URI: {
+                uriList: [String],
+                temporaryDirectoryPath: String,
+                scanners: [String],
+                listener: String,
+                checkMd5: Boolean,
+                encodeUri: Boolean
+            },
+            Local: {
+                listener: String
+            },
+            Custom: {
+                className: String
+            }
+        }
     });
 
     ClusterSchema.index({name: 1, space: 1}, {unique: true});
@@ -843,13 +861,15 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose
             result: {type: String, enum: ['none', 'table', 'bar', 'pie', 'line', 'area']},
             pageSize: Number,
             timeLineSpan: String,
+            maxPages: Number,
             hideSystemColumns: Boolean,
             cacheName: String,
             chartsOptions: {barChart: {stacked: Boolean}, areaChart: {style: String}},
             rate: {
                 value: Number,
                 unit: Number
-            }
+            },
+            qryType: String
         }]
     });
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/backend/routes/demo.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/routes/demo.js b/modules/web-console/backend/routes/demo.js
index ad4be6e..3f4166d 100644
--- a/modules/web-console/backend/routes/demo.js
+++ b/modules/web-console/backend/routes/demo.js
@@ -39,20 +39,17 @@ module.exports.factory = (_, express, settings, mongo, spacesService, errors) =>
         router.post('/reset', (req, res) => {
             spacesService.spaces(req.user._id, true)
                 .then((spaces) => {
-                    if (spaces.length) {
-                        const spaceIds = spaces.map((space) => space._id);
-
-                        return Promise.all([
-                            mongo.Cluster.remove({space: {$in: spaceIds}}).exec(),
-                            mongo.Cache.remove({space: {$in: spaceIds}}).exec(),
-                            mongo.DomainModel.remove({space: {$in: spaceIds}}).exec(),
-                            mongo.Igfs.remove({space: {$in: spaceIds}}).exec()
-                        ]).then(() => spaces[0]);
-                    }
+                    const spaceIds = _.map(spaces, '_id');
+
+                    return spacesService.cleanUp(spaceIds)
+                        .then(() => mongo.Space.remove({_id: {$in: _.tail(spaceIds)}}).exec())
+                        .then(() => _.head(spaces));
                 })
                 .catch((err) => {
                     if (err instanceof errors.MissingResourceException)
                         return spacesService.createDemoSpace(req.user._id);
+
+                    throw err;
                 })
                 .then((space) => {
                     return Promise.all(_.map(clusters, (cluster) => {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/backend/routes/profile.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/routes/profile.js b/modules/web-console/backend/routes/profile.js
index 4d01cda..1d6fccb 100644
--- a/modules/web-console/backend/routes/profile.js
+++ b/modules/web-console/backend/routes/profile.js
@@ -45,7 +45,7 @@ module.exports.factory = function(_, express, mongo, usersService) {
 
             usersService.save(req.body)
                 .then((user) => {
-                    const becomeUsed = req.session.viewedUser && user.admin;
+                    const becomeUsed = req.session.viewedUser && req.user.admin;
 
                     if (becomeUsed) {
                         req.session.viewedUser = user;
@@ -64,6 +64,7 @@ module.exports.factory = function(_, express, mongo, usersService) {
                         });
                     });
                 })
+                .then(() => usersService.get(req.user, req.session.viewedUser))
                 .then(res.api.ok)
                 .catch(res.api.error);
         });

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/backend/services/notebooks.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/services/notebooks.js b/modules/web-console/backend/services/notebooks.js
index 8846d8e..9aa2c38 100644
--- a/modules/web-console/backend/services/notebooks.js
+++ b/modules/web-console/backend/services/notebooks.js
@@ -34,12 +34,14 @@ module.exports = {
 module.exports.factory = (_, mongo, spacesService, errors) => {
     /**
      * Convert remove status operation to own presentation.
+     *
      * @param {RemoveResult} result - The results of remove operation.
      */
     const convertRemoveStatus = ({result}) => ({rowsAffected: result.n});
 
     /**
-     * Update existing notebook
+     * Update existing notebook.
+     *
      * @param {Object} notebook - The notebook for updating
      * @returns {Promise.<mongo.ObjectId>} that resolves cache id
      */
@@ -53,6 +55,7 @@ module.exports.factory = (_, mongo, spacesService, errors) => {
 
     /**
      * Create new notebook.
+     *
      * @param {Object} notebook - The notebook for creation.
      * @returns {Promise.<mongo.ObjectId>} that resolves cache id.
      */
@@ -67,6 +70,7 @@ module.exports.factory = (_, mongo, spacesService, errors) => {
     class NotebooksService {
         /**
          * Create or update Notebook.
+         *
          * @param {Object} notebook - The Notebook
          * @returns {Promise.<mongo.ObjectId>} that resolves Notebook id of merge operation.
          */
@@ -78,16 +82,18 @@ module.exports.factory = (_, mongo, spacesService, errors) => {
         }
 
         /**
-         * Get caches by spaces.
+         * Get notebooks by spaces.
+         *
          * @param {mongo.ObjectId|String} spaceIds - The spaces ids that own caches.
-         * @returns {Promise.<mongo.Cache[]>} - contains requested caches.
+         * @returns {Promise.<mongo.Notebook[]>} - contains requested caches.
          */
         static listBySpaces(spaceIds) {
             return mongo.Notebook.find({space: {$in: spaceIds}}).sort('name').lean().exec();
         }
 
         /**
-         * Remove Notebook.
+         * Remove notebook.
+         *
          * @param {mongo.ObjectId|String} notebookId - The Notebook id for remove.
          * @returns {Promise.<{rowsAffected}>} - The number of affected rows.
          */

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/backend/services/sessions.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/services/sessions.js b/modules/web-console/backend/services/sessions.js
index ff0e303..7f62a60 100644
--- a/modules/web-console/backend/services/sessions.js
+++ b/modules/web-console/backend/services/sessions.js
@@ -38,11 +38,11 @@ module.exports.factory = (_, mongo, errors) => {
          * @param {mongo.ObjectId|String} viewedUserId - id of user to become.
          */
         static become(session, viewedUserId) {
+            if (!session.req.user.admin)
+                return Promise.reject(new errors.IllegalAccessError('Became this user is not permitted. Only administrators can perform this actions.'));
+
             return mongo.Account.findById(viewedUserId).lean().exec()
                 .then((viewedUser) => {
-                    if (!session.req.user.admin)
-                        throw new errors.IllegalAccessError('Became this user is not permitted. Only administrators can perform this actions.');
-
                     viewedUser.token = session.req.user.token;
 
                     session.viewedUser = viewedUser;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/backend/services/spaces.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/services/spaces.js b/modules/web-console/backend/services/spaces.js
index 863d57c..85f346e 100644
--- a/modules/web-console/backend/services/spaces.js
+++ b/modules/web-console/backend/services/spaces.js
@@ -68,6 +68,21 @@ module.exports.factory = (mongo, errors) => {
         static createDemoSpace(userId) {
             return new mongo.Space({name: 'Demo space', owner: userId, demo: true}).save();
         }
+
+        /**
+         * Clean up spaces.
+         *
+         * @param {mongo.ObjectId|String} spaceIds - The space ids for clean up.
+         * @returns {Promise.<>}
+         */
+        static cleanUp(spaceIds) {
+            return Promise.all([
+                mongo.Cluster.remove({space: {$in: spaceIds}}).exec(),
+                mongo.Cache.remove({space: {$in: spaceIds}}).exec(),
+                mongo.DomainModel.remove({space: {$in: spaceIds}}).exec(),
+                mongo.Igfs.remove({space: {$in: spaceIds}}).exec()
+            ]);
+        }
     }
 
     return SpacesService;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/app.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/app.js b/modules/web-console/frontend/app/app.js
index 3510743..4ecd9b5 100644
--- a/modules/web-console/frontend/app/app.js
+++ b/modules/web-console/frontend/app/app.js
@@ -99,11 +99,6 @@ import domainsValidation from './filters/domainsValidation.filter';
 import duration from './filters/duration.filter';
 import hasPojo from './filters/hasPojo.filter';
 
-// Generators
-import $generatorOptional from './modules/configuration/generator/generator-optional';
-
-window.$generatorOptional = $generatorOptional;
-
 // Controllers
 import admin from 'controllers/admin-controller';
 import caches from 'controllers/caches-controller';

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/controllers/reset-password.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/controllers/reset-password.controller.js b/modules/web-console/frontend/app/controllers/reset-password.controller.js
index da0c37b..f84a876 100644
--- a/modules/web-console/frontend/app/controllers/reset-password.controller.js
+++ b/modules/web-console/frontend/app/controllers/reset-password.controller.js
@@ -21,10 +21,10 @@ export default ['resetPassword', [
     ($scope, $modal, $http, $state, Messages, Focus) => {
         if ($state.params.token) {
             $http.post('/api/v1/password/validate/token', {token: $state.params.token})
-                .success((res) => {
-                    $scope.email = res.email;
-                    $scope.token = res.token;
-                    $scope.error = res.error;
+                .then(({data}) => {
+                    $scope.email = data.email;
+                    $scope.token = data.token;
+                    $scope.error = data.error;
 
                     if ($scope.token && !$scope.error)
                         Focus.move('user_password');
@@ -34,16 +34,16 @@ export default ['resetPassword', [
         // Try to reset user password for provided token.
         $scope.resetPassword = (reset_info) => {
             $http.post('/api/v1/password/reset', reset_info)
-                .success(() => {
+                .then(() => {
                     $state.go('signin');
 
                     Messages.showInfo('Password successfully changed');
                 })
-                .error((err, state) => {
+                .catch(({data, state}) => {
                     if (state === 503)
                         $state.go('signin');
 
-                    Messages.showError(err);
+                    Messages.showError(data);
                 });
         };
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/data/event-groups.json
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/data/event-groups.json b/modules/web-console/frontend/app/data/event-groups.json
new file mode 100644
index 0000000..8d0c878
--- /dev/null
+++ b/modules/web-console/frontend/app/data/event-groups.json
@@ -0,0 +1,169 @@
+[
+  {
+    "label": "EVTS_CHECKPOINT",
+    "value": "EVTS_CHECKPOINT",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_CHECKPOINT_SAVED",
+      "EVT_CHECKPOINT_LOADED",
+      "EVT_CHECKPOINT_REMOVED"
+    ]
+  },
+  {
+    "label": "EVTS_DEPLOYMENT",
+    "value": "EVTS_DEPLOYMENT",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_CLASS_DEPLOYED",
+      "EVT_CLASS_UNDEPLOYED",
+      "EVT_CLASS_DEPLOY_FAILED",
+      "EVT_TASK_DEPLOYED",
+      "EVT_TASK_UNDEPLOYED",
+      "EVT_TASK_DEPLOY_FAILED"
+    ]
+  },
+  {
+    "label": "EVTS_ERROR",
+    "value": "EVTS_ERROR",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_JOB_TIMEDOUT",
+      "EVT_JOB_FAILED",
+      "EVT_JOB_FAILED_OVER",
+      "EVT_JOB_REJECTED",
+      "EVT_JOB_CANCELLED",
+      "EVT_TASK_TIMEDOUT",
+      "EVT_TASK_FAILED",
+      "EVT_CLASS_DEPLOY_FAILED",
+      "EVT_TASK_DEPLOY_FAILED",
+      "EVT_TASK_DEPLOYED",
+      "EVT_TASK_UNDEPLOYED",
+      "EVT_CACHE_REBALANCE_STARTED",
+      "EVT_CACHE_REBALANCE_STOPPED"
+    ]
+  },
+  {
+    "label": "EVTS_DISCOVERY",
+    "value": "EVTS_DISCOVERY",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_NODE_JOINED",
+      "EVT_NODE_LEFT",
+      "EVT_NODE_FAILED",
+      "EVT_NODE_SEGMENTED",
+      "EVT_CLIENT_NODE_DISCONNECTED",
+      "EVT_CLIENT_NODE_RECONNECTED"
+    ]
+  },
+  {
+    "label": "EVTS_JOB_EXECUTION",
+    "value": "EVTS_JOB_EXECUTION",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_JOB_MAPPED",
+      "EVT_JOB_RESULTED",
+      "EVT_JOB_FAILED_OVER",
+      "EVT_JOB_STARTED",
+      "EVT_JOB_FINISHED",
+      "EVT_JOB_TIMEDOUT",
+      "EVT_JOB_REJECTED",
+      "EVT_JOB_FAILED",
+      "EVT_JOB_QUEUED",
+      "EVT_JOB_CANCELLED"
+    ]
+  },
+  {
+    "label": "EVTS_TASK_EXECUTION",
+    "value": "EVTS_TASK_EXECUTION",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_TASK_STARTED",
+      "EVT_TASK_FINISHED",
+      "EVT_TASK_FAILED",
+      "EVT_TASK_TIMEDOUT",
+      "EVT_TASK_SESSION_ATTR_SET",
+      "EVT_TASK_REDUCED"
+    ]
+  },
+  {
+    "label": "EVTS_CACHE",
+    "value": "EVTS_CACHE",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_CACHE_ENTRY_CREATED",
+      "EVT_CACHE_ENTRY_DESTROYED",
+      "EVT_CACHE_OBJECT_PUT",
+      "EVT_CACHE_OBJECT_READ",
+      "EVT_CACHE_OBJECT_REMOVED",
+      "EVT_CACHE_OBJECT_LOCKED",
+      "EVT_CACHE_OBJECT_UNLOCKED",
+      "EVT_CACHE_OBJECT_SWAPPED",
+      "EVT_CACHE_OBJECT_UNSWAPPED",
+      "EVT_CACHE_OBJECT_EXPIRED"
+    ]
+  },
+  {
+    "label": "EVTS_CACHE_REBALANCE",
+    "value": "EVTS_CACHE_REBALANCE",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_CACHE_REBALANCE_STARTED",
+      "EVT_CACHE_REBALANCE_STOPPED",
+      "EVT_CACHE_REBALANCE_PART_LOADED",
+      "EVT_CACHE_REBALANCE_PART_UNLOADED",
+      "EVT_CACHE_REBALANCE_OBJECT_LOADED",
+      "EVT_CACHE_REBALANCE_OBJECT_UNLOADED",
+      "EVT_CACHE_REBALANCE_PART_DATA_LOST"
+    ]
+  },
+  {
+    "label": "EVTS_CACHE_LIFECYCLE",
+    "value": "EVTS_CACHE_LIFECYCLE",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_CACHE_STARTED",
+      "EVT_CACHE_STOPPED",
+      "EVT_CACHE_NODES_LEFT"
+    ]
+  },
+  {
+    "label": "EVTS_CACHE_QUERY",
+    "value": "EVTS_CACHE_QUERY",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_CACHE_QUERY_EXECUTED",
+      "EVT_CACHE_QUERY_OBJECT_READ"
+    ]
+  },
+  {
+    "label": "EVTS_SWAPSPACE",
+    "value": "EVTS_SWAPSPACE",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_SWAP_SPACE_CLEARED",
+      "EVT_SWAP_SPACE_DATA_REMOVED",
+      "EVT_SWAP_SPACE_DATA_READ",
+      "EVT_SWAP_SPACE_DATA_STORED",
+      "EVT_SWAP_SPACE_DATA_EVICTED"
+    ]
+  },
+  {
+    "label": "EVTS_IGFS",
+    "value": "EVTS_IGFS",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_IGFS_FILE_CREATED",
+      "EVT_IGFS_FILE_RENAMED",
+      "EVT_IGFS_FILE_DELETED",
+      "EVT_IGFS_FILE_OPENED_READ",
+      "EVT_IGFS_FILE_OPENED_WRITE",
+      "EVT_IGFS_FILE_CLOSED_WRITE",
+      "EVT_IGFS_FILE_CLOSED_READ",
+      "EVT_IGFS_FILE_PURGED",
+      "EVT_IGFS_META_UPDATED",
+      "EVT_IGFS_DIR_CREATED",
+      "EVT_IGFS_DIR_RENAMED",
+      "EVT_IGFS_DIR_DELETED"
+    ]
+  }
+]

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/data/event-types.json
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/data/event-types.json b/modules/web-console/frontend/app/data/event-types.json
deleted file mode 100644
index 8d0c878..0000000
--- a/modules/web-console/frontend/app/data/event-types.json
+++ /dev/null
@@ -1,169 +0,0 @@
-[
-  {
-    "label": "EVTS_CHECKPOINT",
-    "value": "EVTS_CHECKPOINT",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_CHECKPOINT_SAVED",
-      "EVT_CHECKPOINT_LOADED",
-      "EVT_CHECKPOINT_REMOVED"
-    ]
-  },
-  {
-    "label": "EVTS_DEPLOYMENT",
-    "value": "EVTS_DEPLOYMENT",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_CLASS_DEPLOYED",
-      "EVT_CLASS_UNDEPLOYED",
-      "EVT_CLASS_DEPLOY_FAILED",
-      "EVT_TASK_DEPLOYED",
-      "EVT_TASK_UNDEPLOYED",
-      "EVT_TASK_DEPLOY_FAILED"
-    ]
-  },
-  {
-    "label": "EVTS_ERROR",
-    "value": "EVTS_ERROR",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_JOB_TIMEDOUT",
-      "EVT_JOB_FAILED",
-      "EVT_JOB_FAILED_OVER",
-      "EVT_JOB_REJECTED",
-      "EVT_JOB_CANCELLED",
-      "EVT_TASK_TIMEDOUT",
-      "EVT_TASK_FAILED",
-      "EVT_CLASS_DEPLOY_FAILED",
-      "EVT_TASK_DEPLOY_FAILED",
-      "EVT_TASK_DEPLOYED",
-      "EVT_TASK_UNDEPLOYED",
-      "EVT_CACHE_REBALANCE_STARTED",
-      "EVT_CACHE_REBALANCE_STOPPED"
-    ]
-  },
-  {
-    "label": "EVTS_DISCOVERY",
-    "value": "EVTS_DISCOVERY",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_NODE_JOINED",
-      "EVT_NODE_LEFT",
-      "EVT_NODE_FAILED",
-      "EVT_NODE_SEGMENTED",
-      "EVT_CLIENT_NODE_DISCONNECTED",
-      "EVT_CLIENT_NODE_RECONNECTED"
-    ]
-  },
-  {
-    "label": "EVTS_JOB_EXECUTION",
-    "value": "EVTS_JOB_EXECUTION",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_JOB_MAPPED",
-      "EVT_JOB_RESULTED",
-      "EVT_JOB_FAILED_OVER",
-      "EVT_JOB_STARTED",
-      "EVT_JOB_FINISHED",
-      "EVT_JOB_TIMEDOUT",
-      "EVT_JOB_REJECTED",
-      "EVT_JOB_FAILED",
-      "EVT_JOB_QUEUED",
-      "EVT_JOB_CANCELLED"
-    ]
-  },
-  {
-    "label": "EVTS_TASK_EXECUTION",
-    "value": "EVTS_TASK_EXECUTION",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_TASK_STARTED",
-      "EVT_TASK_FINISHED",
-      "EVT_TASK_FAILED",
-      "EVT_TASK_TIMEDOUT",
-      "EVT_TASK_SESSION_ATTR_SET",
-      "EVT_TASK_REDUCED"
-    ]
-  },
-  {
-    "label": "EVTS_CACHE",
-    "value": "EVTS_CACHE",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_CACHE_ENTRY_CREATED",
-      "EVT_CACHE_ENTRY_DESTROYED",
-      "EVT_CACHE_OBJECT_PUT",
-      "EVT_CACHE_OBJECT_READ",
-      "EVT_CACHE_OBJECT_REMOVED",
-      "EVT_CACHE_OBJECT_LOCKED",
-      "EVT_CACHE_OBJECT_UNLOCKED",
-      "EVT_CACHE_OBJECT_SWAPPED",
-      "EVT_CACHE_OBJECT_UNSWAPPED",
-      "EVT_CACHE_OBJECT_EXPIRED"
-    ]
-  },
-  {
-    "label": "EVTS_CACHE_REBALANCE",
-    "value": "EVTS_CACHE_REBALANCE",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_CACHE_REBALANCE_STARTED",
-      "EVT_CACHE_REBALANCE_STOPPED",
-      "EVT_CACHE_REBALANCE_PART_LOADED",
-      "EVT_CACHE_REBALANCE_PART_UNLOADED",
-      "EVT_CACHE_REBALANCE_OBJECT_LOADED",
-      "EVT_CACHE_REBALANCE_OBJECT_UNLOADED",
-      "EVT_CACHE_REBALANCE_PART_DATA_LOST"
-    ]
-  },
-  {
-    "label": "EVTS_CACHE_LIFECYCLE",
-    "value": "EVTS_CACHE_LIFECYCLE",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_CACHE_STARTED",
-      "EVT_CACHE_STOPPED",
-      "EVT_CACHE_NODES_LEFT"
-    ]
-  },
-  {
-    "label": "EVTS_CACHE_QUERY",
-    "value": "EVTS_CACHE_QUERY",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_CACHE_QUERY_EXECUTED",
-      "EVT_CACHE_QUERY_OBJECT_READ"
-    ]
-  },
-  {
-    "label": "EVTS_SWAPSPACE",
-    "value": "EVTS_SWAPSPACE",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_SWAP_SPACE_CLEARED",
-      "EVT_SWAP_SPACE_DATA_REMOVED",
-      "EVT_SWAP_SPACE_DATA_READ",
-      "EVT_SWAP_SPACE_DATA_STORED",
-      "EVT_SWAP_SPACE_DATA_EVICTED"
-    ]
-  },
-  {
-    "label": "EVTS_IGFS",
-    "value": "EVTS_IGFS",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_IGFS_FILE_CREATED",
-      "EVT_IGFS_FILE_RENAMED",
-      "EVT_IGFS_FILE_DELETED",
-      "EVT_IGFS_FILE_OPENED_READ",
-      "EVT_IGFS_FILE_OPENED_WRITE",
-      "EVT_IGFS_FILE_CLOSED_WRITE",
-      "EVT_IGFS_FILE_CLOSED_READ",
-      "EVT_IGFS_FILE_PURGED",
-      "EVT_IGFS_META_UPDATED",
-      "EVT_IGFS_DIR_CREATED",
-      "EVT_IGFS_DIR_RENAMED",
-      "EVT_IGFS_DIR_DELETED"
-    ]
-  }
-]

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/data/pom-dependencies.json
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/data/pom-dependencies.json b/modules/web-console/frontend/app/data/pom-dependencies.json
index acf2bc8..7d2bed0 100644
--- a/modules/web-console/frontend/app/data/pom-dependencies.json
+++ b/modules/web-console/frontend/app/data/pom-dependencies.json
@@ -10,11 +10,11 @@
     "HadoopIgfsJcl": {"artifactId": "ignite-hadoop"},
     "SLF4J": {"artifactId": "ignite-slf4j"},
 
-    "Generic": {"groupId": "com.mchange", "artifactId": "c3p0", "version": "0.9.5.1"},
-    "MySQL": {"groupId": "mysql", "artifactId": "mysql-connector-java", "version": "5.1.37"},
-    "PostgreSQL": {"groupId": "org.postgresql", "artifactId": "postgresql", "version": "9.4-1204-jdbc42"},
+    "Generic": {"groupId": "com.mchange", "artifactId": "c3p0", "version": "0.9.5.2"},
+    "MySQL": {"groupId": "mysql", "artifactId": "mysql-connector-java", "version": "5.1.40"},
+    "PostgreSQL": {"groupId": "org.postgresql", "artifactId": "postgresql", "version": "9.4.1212.jre7"},
     "H2": {"groupId": "com.h2database", "artifactId": "h2", "version": "1.4.191"},
-    "Oracle": {"groupId": "oracle", "artifactId": "jdbc", "version": "11.2", "jar": "ojdbc6.jar"},
-    "DB2": {"groupId": "ibm", "artifactId": "jdbc", "version": "4.19.26", "jar": "db2jcc4.jar"},
-    "SQLServer": {"groupId": "microsoft", "artifactId": "jdbc", "version": "4.1", "jar": "sqljdbc41.jar"}
+    "Oracle": {"groupId": "com.oracle.jdbc", "artifactId": "ojdbc7", "version": "12.1.0.2", "jar": "ojdbc7.jar"},
+    "DB2": {"groupId": "ibm", "artifactId": "jdbc", "version": "4.21.29", "jar": "db2jcc4.jar"},
+    "SQLServer": {"groupId": "microsoft", "artifactId": "jdbc", "version": "4.2", "jar": "sqljdbc41.jar"}
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.controller.js b/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.controller.js
index 32feaf3..de335ae 100644
--- a/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.controller.js
+++ b/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.controller.js
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-export default ['$scope', 'GeneratorDocker', function($scope, docker) {
+export default ['$scope', 'IgniteDockerGenerator', function($scope, docker) {
     const ctrl = this;
 
     // Watchers definition.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.jade b/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.jade
index 3b0e7b8..3a24cfb 100644
--- a/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.jade
+++ b/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.jade
@@ -20,7 +20,7 @@ mixin hard-link(ref, txt)
 .panel-details-noborder
     .details-row
         p
-            +hard-link('https://docs.docker.com/reference/builder', 'Docker')
+            +hard-link('https://docs.docker.com/engine/reference/builder/', 'Docker')
             | &nbsp;file is a text file with instructions to create Docker image.<br/>
             | To build image you have to store following Docker file with your Ignite XML configuration to the same directory.<br>
             | Also you could use predefined&nbsp;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js b/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js
index 4e11874..61bf086 100644
--- a/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js
+++ b/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js
@@ -30,7 +30,7 @@ export default ['$scope', 'JavaTypes', 'JavaTransformer', function($scope, JavaT
     const updatePojos = () => {
         delete ctrl.pojos;
 
-        if (!ctrl.cluster || !ctrl.cluster.caches)
+        if (_.isNil(ctrl.cluster) || _.isEmpty(ctrl.cluster.caches))
             return;
 
         ctrl.pojos = generator.pojos(ctrl.cluster.caches, ctrl.useConstructor, ctrl.includeKeyFields);
@@ -46,7 +46,7 @@ export default ['$scope', 'JavaTypes', 'JavaTransformer', function($scope, JavaT
         const classes = ctrl.classes = [];
 
         _.forEach(ctrl.pojos, (pojo) => {
-            if (pojo.keyType && JavaTypes.nonBuiltInClass(pojo.keyType))
+            if (_.nonNil(pojo.keyClass))
                 classes.push(pojo.keyType);
 
             classes.push(pojo.valueType);
@@ -55,17 +55,17 @@ export default ['$scope', 'JavaTypes', 'JavaTransformer', function($scope, JavaT
 
     // Update pojos class.
     const updateClass = (value) => {
-        if (!value || !ctrl.pojos.length)
+        if (_.isEmpty(value))
             return;
 
-        const keyType = ctrl.pojos[0].keyType;
+        const pojo = value[0];
 
-        ctrl.class = ctrl.class || (JavaTypes.nonBuiltInClass(keyType) ? keyType : null) || ctrl.pojos[0].valueType;
+        ctrl.class = ctrl.class || (pojo.keyClass ? pojo.keyType : pojo.valueType);
     };
 
     // Update pojos data.
     const updatePojosData = (value) => {
-        if (!value)
+        if (_.isNil(value))
             return;
 
         _.forEach(ctrl.pojos, (pojo) => {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js b/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js
index 2bf78c3..477cf20 100644
--- a/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js
+++ b/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-export default ['$scope', 'GeneratorPom', 'IgniteVersion', function($scope, pom, Version) {
+export default ['$scope', 'IgniteMavenGenerator', 'IgniteVersion', function($scope, maven, Version) {
     const ctrl = this;
 
     // Watchers definition.
@@ -25,7 +25,7 @@ export default ['$scope', 'GeneratorPom', 'IgniteVersion', function($scope, pom,
         if (!value)
             return;
 
-        ctrl.data = pom.generate($scope.cluster, Version.productVersion().ignite).asString();
+        ctrl.data = maven.generate($scope.cluster, Version.productVersion().ignite).asString();
     };
 
     // Setup watchers.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade
index 298db52..33af6d1 100644
--- a/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade
+++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade
@@ -28,7 +28,7 @@ mixin ignite-form-field-dropdown(label, model, name, disabled, required, multipl
             data-ng-disabled=disabled && '#{disabled}' || '!#{options}.length'
 
             bs-select
-            bs-options='item.value as item.label for item in #{options}' 
+            bs-options='item.value as item.label for item in #{options}'
 
             data-multiple=multiple ? '1' : false
             data-container='body > .wrapper'
@@ -41,7 +41,8 @@ mixin ignite-form-field-dropdown(label, model, name, disabled, required, multipl
     .ignite-form-field
         +ignite-form-field__label(label, name, required)
         .ignite-form-field__control
-            i.tipField.icon-help(bs-tooltip='' data-title=tip)
+            if tip
+                i.tipField.icon-help(bs-tooltip='' data-title=tip)
 
             if block
                 block

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade
index d48343c..58b0dcd 100644
--- a/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade
+++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade
@@ -38,7 +38,8 @@ mixin ignite-form-field-number(label, model, name, disabled, required, placehold
     .ignite-form-field
         +ignite-form-field__label(label, name, required)
         .ignite-form-field__control
-            i.tipField.icon-help(bs-tooltip='' data-title=tip)
+            if tip
+                i.tipField.icon-help(bs-tooltip='' data-title=tip)
             
             +form-field-feedback(name, 'required', 'This field could not be empty')
             +form-field-feedback(name, 'min', 'Value is less than allowable minimum: '+ min || 0)

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade
index 136d23b..1f93d3b 100644
--- a/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade
+++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade
@@ -30,13 +30,30 @@ mixin ignite-form-field-input(name, model, disabled, required, placeholder)
         data-ignite-form-panel-field=''
     )&attributes(attributes ? attributes.attributes ? attributes.attributes : attributes: {})
 
+mixin ignite-form-field-url-input(name, model, disabled, required, placeholder)
+    input.form-control(
+        id='{{ #{name} }}Input'
+        name='{{ #{name} }}'
+        placeholder=placeholder
+        type='url'
+
+        data-ng-model=model
+
+        data-ng-required=required && '#{required}'
+        data-ng-disabled=disabled && '#{disabled}'
+        data-ng-focus='tableReset()'
+
+        data-ignite-form-panel-field=''
+    )&attributes(attributes ? attributes.attributes ? attributes.attributes : attributes: {})
+
 mixin ignite-form-field-text(label, model, name, disabled, required, placeholder, tip)
     -var errLbl = label.substring(0, label.length - 1)
 
     .ignite-form-field
         +ignite-form-field__label(label, name, required)
         .ignite-form-field__control
-            i.tipField.icon-help(bs-tooltip='' data-title=tip)
+            if tip
+                i.tipField.icon-help(bs-tooltip='' data-title=tip)
             
             if block
                 block

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/helpers/jade/mixins.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/helpers/jade/mixins.jade b/modules/web-console/frontend/app/helpers/jade/mixins.jade
index 92af1b0..6ca41f6 100644
--- a/modules/web-console/frontend/app/helpers/jade/mixins.jade
+++ b/modules/web-console/frontend/app/helpers/jade/mixins.jade
@@ -183,6 +183,14 @@ mixin text-enabled(lbl, model, name, enabled, required, placeholder, tip)
         if  block
             block
 
+//- Mixin for text field with autofocus.
+mixin text-enabled-autofocus(lbl, model, name, enabled, required, placeholder, tip)
+    +ignite-form-field-text(lbl, model, name, enabledToDisabled(enabled), required, placeholder, tip)(
+        data-ignite-form-field-input-autofocus='true'
+    )
+        if  block
+            block
+
 //- Mixin for text field.
 mixin text(lbl, model, name, required, placeholder, tip)
     +ignite-form-field-text(lbl, model, name, false, required, placeholder, tip)
@@ -221,12 +229,28 @@ mixin dropdown-required-empty(lbl, model, name, enabled, required, placeholder,
         if  block
             block
 
+//- Mixin for required dropdown field with autofocus.
+mixin dropdown-required-empty-autofocus(lbl, model, name, enabled, required, placeholder, placeholderEmpty, options, tip)
+    +ignite-form-field-dropdown(lbl, model, name, enabledToDisabled(enabled), required, false, placeholder, placeholderEmpty, options, tip)(
+        data-ignite-form-field-input-autofocus='true'
+    )
+        if  block
+            block
+
 //- Mixin for required dropdown field.
 mixin dropdown-required(lbl, model, name, enabled, required, placeholder, options, tip)
     +ignite-form-field-dropdown(lbl, model, name, enabledToDisabled(enabled), required, false, placeholder, '', options, tip)
         if  block
             block
 
+//- Mixin for required dropdown field with autofocus.
+mixin dropdown-required-autofocus(lbl, model, name, enabled, required, placeholder, options, tip)
+    +ignite-form-field-dropdown(lbl, model, name, enabledToDisabled(enabled), required, false, placeholder, '', options, tip)(
+        data-ignite-form-field-input-autofocus='true'
+    )
+        if  block
+            block
+
 //- Mixin for dropdown field.
 mixin dropdown(lbl, model, name, enabled, placeholder, options, tip)
     +ignite-form-field-dropdown(lbl, model, name, enabledToDisabled(enabled), false, false, placeholder, '', options, tip)
@@ -324,6 +348,28 @@ mixin table-java-package-field(name, model, items, valid, save, newItem)
                 ignite-on-escape=onEscape
             )
 
+//- Mixin for table java package field.
+mixin table-url-field(name, model, items, valid, save, newItem)
+    -var resetOnEnter = newItem ? '(stopblur = true) && (group.add = [{}])' : '(field.edit = false)'
+    -var onEnter = valid + ' && (' + save + '); ' + valid + ' && ' + resetOnEnter + ';'
+
+    -var onEscape = newItem ? 'group.add = []' : 'field.edit = false'
+
+    -var resetOnBlur = newItem ? '!stopblur && (group.add = [])' : 'field.edit = false'
+    -var onBlur = valid + ' && ( ' + save + '); ' + resetOnBlur + ';'
+
+    div(ignite-on-focus-out=onBlur)
+        if block
+            block
+
+        .input-tip
+            +ignite-form-field-url-input(name, model, false, 'true', 'Enter URL')(
+                data-ignite-unique=items
+                data-ignite-form-field-input-autofocus='true'
+
+                ignite-on-enter=onEnter
+                ignite-on-escape=onEscape
+            )
 
 //- Mixin for table address field.
 mixin table-address-field(name, model, items, valid, save, newItem, portRange)
@@ -393,17 +439,17 @@ mixin table-save-button(valid, save, newItem)
     )
 
 //- Mixin for table remove button.
-mixin table-remove-conditional-button(items, show, tip)
+mixin table-remove-conditional-button(items, show, tip, row)
     i.tipField.fa.fa-remove(
         ng-hide='!#{show} || field.edit'
         bs-tooltip
         data-title=tip
-        ng-click='#{items}.splice(#{items}.indexOf(model), 1)'
+        ng-click='#{items}.splice(#{items}.indexOf(#{row}), 1)'
     )
 
 //- Mixin for table remove button.
 mixin table-remove-button(items, tip)
-    +table-remove-conditional-button(items, 'true', tip)
+    +table-remove-conditional-button(items, 'true', tip, 'model')
 
 //- Mixin for cache mode.
 mixin cacheMode(lbl, model, name, placeholder)

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/Demo/Demo.module.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/Demo/Demo.module.js b/modules/web-console/frontend/app/modules/Demo/Demo.module.js
index 83d55ed..a3700ca 100644
--- a/modules/web-console/frontend/app/modules/Demo/Demo.module.js
+++ b/modules/web-console/frontend/app/modules/Demo/Demo.module.js
@@ -41,11 +41,11 @@ angular
             url: '/demo/reset',
             controller: ['$state', '$http', 'IgniteMessages', ($state, $http, Messages) => {
                 $http.post('/api/v1/demo/reset')
-                    .success(() => $state.go('base.configuration.clusters'))
-                    .error((err) => {
+                    .then(() => $state.go('base.configuration.clusters'))
+                    .catch((res) => {
                         $state.go('base.configuration.clusters');
 
-                        Messages.showError(err);
+                        Messages.showError(res);
                     });
             }],
             metaTags: {}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/EventGroups.provider.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/EventGroups.provider.js b/modules/web-console/frontend/app/modules/configuration/EventGroups.provider.js
deleted file mode 100644
index 61f3188..0000000
--- a/modules/web-console/frontend/app/modules/configuration/EventGroups.provider.js
+++ /dev/null
@@ -1,30 +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.
- */
-
-// Events groups.
-import GROUPS from 'app/data/event-types.json';
-
-export default ['igniteEventGroups', function() {
-    const groups = GROUPS;
-
-    this.push = (data) => groups.push(data);
-
-    this.$get = [() => {
-        return groups;
-    }];
-}];
-

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/Version.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/Version.service.js b/modules/web-console/frontend/app/modules/configuration/Version.service.js
index 06efdda..f0e9c4c 100644
--- a/modules/web-console/frontend/app/modules/configuration/Version.service.js
+++ b/modules/web-console/frontend/app/modules/configuration/Version.service.js
@@ -22,7 +22,7 @@ const VERSION_MATCHER = /(\d+)\.(\d+)\.(\d+)([-.]([^0123456789][^-]+)(-SNAPSHOT)
 
 const numberComparator = (a, b) => a > b ? 1 : a < b ? -1 : 0;
 
-export default class Version {
+export default class IgniteVersion {
     /**
      * Tries to parse product version from it's string representation.
      *
@@ -70,7 +70,7 @@ export default class Version {
         if (res !== 0)
             return res;
 
-        return numberComparator(pa.revTs, pb.maintenance);
+        return numberComparator(pa.revTs, pb.revTs);
     }
 
     /**
@@ -79,7 +79,7 @@ export default class Version {
      */
     productVersion() {
         return {
-            ignite: '1.7.0'
+            ignite: '1.8.0'
         };
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/configuration.module.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/configuration.module.js b/modules/web-console/frontend/app/modules/configuration/configuration.module.js
index 27f7bef..4288ff7 100644
--- a/modules/web-console/frontend/app/modules/configuration/configuration.module.js
+++ b/modules/web-console/frontend/app/modules/configuration/configuration.module.js
@@ -17,26 +17,28 @@
 
 import angular from 'angular';
 
-import igniteEventGroups from './EventGroups.provider';
+
 import igniteSidebar from './Sidebar.provider';
-import Version from './Version.service';
+import IgniteVersion from './Version.service';
 
-import clusterDefaults from './generator/defaults/cluster.provider';
-import clusterPlatformDefaults from './generator/defaults/cluster.platform.provider';
-import cacheDefaults from './generator/defaults/cache.provider';
-import cachePlatformDefaults from './generator/defaults/cache.platform.provider';
-import igfsDefaults from './generator/defaults/igfs.provider';
+import IgniteClusterDefaults from './generator/defaults/Cluster.service';
+import IgniteClusterPlatformDefaults from './generator/defaults/Cluster.platform.service';
+import IgniteCacheDefaults from './generator/defaults/Cache.service';
+import IgniteCachePlatformDefaults from './generator/defaults/Cache.platform.service';
+import IgniteIGFSDefaults from './generator/defaults/IGFS.service';
+import IgniteEventGroups from './generator/defaults/Event-groups.service';
 
-import ConfigurationGenerator from './generator/ConfigurationGenerator';
-import PlatformGenerator from './generator/PlatformGenerator';
+import IgniteConfigurationGenerator from './generator/ConfigurationGenerator';
+import IgnitePlatformGenerator from './generator/PlatformGenerator';
 
-import SpringTransformer from './generator/SpringTransformer.service';
-import JavaTransformer from './generator/JavaTransformer.service';
+import IgniteSpringTransformer from './generator/SpringTransformer.service';
+import IgniteJavaTransformer from './generator/JavaTransformer.service';
 import SharpTransformer from './generator/SharpTransformer.service';
-import GeneratorDocker from './generator/Docker.service';
-import GeneratorPom from './generator/Pom.service';
-import GeneratorProperties from './generator/Properties.service';
-import GeneratorReadme from './generator/Readme.service';
+import IgniteDockerGenerator from './generator/Docker.service';
+import IgniteMavenGenerator from './generator/Maven.service';
+import IgniteGeneratorProperties from './generator/Properties.service';
+import IgniteReadmeGenerator from './generator/Readme.service';
+import IgniteCustomGenerator from './generator/Custom.service';
 
 import igniteSidebarDirective from './sidebar.directive';
 
@@ -45,21 +47,22 @@ angular
 .module('ignite-console.configuration', [
 
 ])
-.provider('igniteClusterDefaults', clusterDefaults)
-.provider('igniteClusterPlatformDefaults', clusterPlatformDefaults)
-.provider('igniteCacheDefaults', cacheDefaults)
-.provider('igniteCachePlatformDefaults', cachePlatformDefaults)
-.provider('igniteIgfsDefaults', igfsDefaults)
-.provider(...igniteEventGroups)
 .provider(...igniteSidebar)
 .directive(...igniteSidebarDirective)
-.service('IgniteVersion', Version)
-.service('IgniteConfigurationGenerator', ConfigurationGenerator)
-.service('IgnitePlatformGenerator', PlatformGenerator)
-.service('SpringTransformer', SpringTransformer)
-.service('JavaTransformer', JavaTransformer)
+.service('IgniteConfigurationGenerator', IgniteConfigurationGenerator)
+.service('IgnitePlatformGenerator', IgnitePlatformGenerator)
+.service('SpringTransformer', IgniteSpringTransformer)
+.service('JavaTransformer', IgniteJavaTransformer)
 .service('IgniteSharpTransformer', SharpTransformer)
-.service('IgnitePropertiesGenerator', GeneratorProperties)
-.service('IgniteReadmeGenerator', GeneratorReadme)
-.service(...GeneratorDocker)
-.service(...GeneratorPom);
+.service('IgniteVersion', IgniteVersion)
+.service('IgniteEventGroups', IgniteEventGroups)
+.service('IgniteClusterDefaults', IgniteClusterDefaults)
+.service('IgniteClusterPlatformDefaults', IgniteClusterPlatformDefaults)
+.service('IgniteCacheDefaults', IgniteCacheDefaults)
+.service('IgniteCachePlatformDefaults', IgniteCachePlatformDefaults)
+.service('IgniteIGFSDefaults', IgniteIGFSDefaults)
+.service('IgnitePropertiesGenerator', IgniteGeneratorProperties)
+.service('IgniteReadmeGenerator', IgniteReadmeGenerator)
+.service('IgniteDockerGenerator', IgniteDockerGenerator)
+.service('IgniteMavenGenerator', IgniteMavenGenerator)
+.service('IgniteCustomGenerator', IgniteCustomGenerator);

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js b/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js
index 6244a53..f5afe59 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js
@@ -17,7 +17,24 @@
 
 import StringBuilder from './StringBuilder';
 
+import IgniteConfigurationGenerator from './ConfigurationGenerator';
+import IgniteEventGroups from './defaults/Event-groups.service';
+
+import IgniteClusterDefaults from './defaults/Cluster.service';
+import IgniteCacheDefaults from './defaults/Cache.service';
+import IgniteIGFSDefaults from './defaults/IGFS.service';
+
+import JavaTypes from '../../../services/JavaTypes.service';
+
+const clusterDflts = new IgniteClusterDefaults();
+const cacheDflts = new IgniteCacheDefaults();
+const igfsDflts = new IgniteIGFSDefaults();
+
 export default class AbstractTransformer {
+    static generator = IgniteConfigurationGenerator;
+    static javaTypes = new JavaTypes(clusterDflts, cacheDflts, igfsDflts);
+    static eventGroups = new IgniteEventGroups();
+
     // Append comment with time stamp.
     static mainComment(sb, ...lines) {
         lines.push(sb.generatedBy());

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/Beans.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Beans.js b/modules/web-console/frontend/app/modules/configuration/generator/Beans.js
index 2750626..ca19342 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/Beans.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/Beans.js
@@ -17,6 +17,11 @@
 
 import _ from 'lodash';
 
+_.mixin({
+    nonNil: _.negate(_.isNil),
+    nonEmpty: _.negate(_.isEmpty)
+});
+
 export class EmptyBean {
     /**
      * @param {String} clsName


[33/50] [abbrv] ignite git commit: Web console beta-7.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
index 5887832..8770bf6 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
@@ -19,1776 +19,1825 @@ import DFLT_DIALECTS from 'app/data/dialects.json';
 
 import { EmptyBean, Bean } from './Beans';
 
-export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'igniteIgfsDefaults', (JavaTypes, clusterDflts, cacheDflts, igfsDflts) => {
-    class ConfigurationGenerator {
-        static igniteConfigurationBean(cluster) {
-            return new Bean('org.apache.ignite.configuration.IgniteConfiguration', 'cfg', cluster, clusterDflts);
-        }
+import IgniteClusterDefaults from './defaults/Cluster.service';
+import IgniteCacheDefaults from './defaults/Cache.service';
+import IgniteIGFSDefaults from './defaults/IGFS.service';
 
-        static igfsConfigurationBean(igfs) {
-            return new Bean('org.apache.ignite.configuration.FileSystemConfiguration', 'igfs', igfs, igfsDflts);
-        }
+import JavaTypes from '../../../services/JavaTypes.service';
 
-        static cacheConfigurationBean(cache) {
-            return new Bean('org.apache.ignite.configuration.CacheConfiguration', 'ccfg', cache, cacheDflts);
-        }
+const clusterDflts = new IgniteClusterDefaults();
+const cacheDflts = new IgniteCacheDefaults();
+const igfsDflts = new IgniteIGFSDefaults();
 
-        static domainConfigurationBean(domain) {
-            return new Bean('org.apache.ignite.cache.QueryEntity', 'qryEntity', domain, cacheDflts);
-        }
+const javaTypes = new JavaTypes(clusterDflts, cacheDflts, igfsDflts);
 
-        static discoveryConfigurationBean(discovery) {
-            return new Bean('org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi', 'discovery', discovery, clusterDflts.discovery);
-        }
+export default class IgniteConfigurationGenerator {
+    static igniteConfigurationBean(cluster) {
+        return new Bean('org.apache.ignite.configuration.IgniteConfiguration', 'cfg', cluster, clusterDflts);
+    }
 
-        /**
-         * Function to generate ignite configuration.
-         *
-         * @param {Object} cluster Cluster to process.
-         * @param {Boolean} client
-         * @return {Bean} Generated ignite configuration.
-         */
-        static igniteConfiguration(cluster, client) {
-            const cfg = this.igniteConfigurationBean(cluster);
-
-            this.clusterGeneral(cluster, cfg, client);
-            this.clusterAtomics(cluster.atomicConfiguration, cfg);
-            this.clusterBinary(cluster.binaryConfiguration, cfg);
-            this.clusterCacheKeyConfiguration(cluster.cacheKeyConfiguration, cfg);
-            this.clusterCheckpoint(cluster, cluster.caches, cfg);
-            this.clusterCollision(cluster.collision, cfg);
-            this.clusterCommunication(cluster, cfg);
-            this.clusterConnector(cluster.connector, cfg);
-            this.clusterDeployment(cluster, cfg);
-            this.clusterEvents(cluster, cfg);
-            this.clusterFailover(cluster, cfg);
-            this.clusterLoadBalancing(cluster, cfg);
-            this.clusterLogger(cluster.logger, cfg);
-            this.clusterODBC(cluster.odbc, cfg);
-            this.clusterMarshaller(cluster, cfg);
-            this.clusterMetrics(cluster, cfg);
-            this.clusterSwap(cluster, cfg);
-            this.clusterTime(cluster, cfg);
-            this.clusterPools(cluster, cfg);
-            this.clusterTransactions(cluster.transactionConfiguration, cfg);
-            this.clusterSsl(cluster, cfg);
-            this.clusterUserAttributes(cluster, cfg);
-
-            this.clusterCaches(cluster, cluster.caches, cluster.igfss, client, cfg);
-
-            if (!client)
-                this.clusterIgfss(cluster.igfss, cfg);
+    static igfsConfigurationBean(igfs) {
+        return new Bean('org.apache.ignite.configuration.FileSystemConfiguration', 'igfs', igfs, igfsDflts);
+    }
 
-            return cfg;
-        }
+    static cacheConfigurationBean(cache) {
+        return new Bean('org.apache.ignite.configuration.CacheConfiguration', 'ccfg', cache, cacheDflts);
+    }
 
-        static dialectClsName(dialect) {
-            return DFLT_DIALECTS[dialect] || 'Unknown database: ' + (dialect || 'Choose JDBC dialect');
-        }
+    static domainConfigurationBean(domain) {
+        return new Bean('org.apache.ignite.cache.QueryEntity', 'qryEntity', domain, cacheDflts);
+    }
 
-        static dataSourceBean(id, dialect) {
-            let dsBean;
+    static discoveryConfigurationBean(discovery) {
+        return new Bean('org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi', 'discovery', discovery, clusterDflts.discovery);
+    }
 
-            switch (dialect) {
-                case 'Generic':
-                    dsBean = new Bean('com.mchange.v2.c3p0.ComboPooledDataSource', id, {})
-                        .property('jdbcUrl', `${id}.jdbc.url`, 'jdbc:your_database');
+    /**
+     * Function to generate ignite configuration.
+     *
+     * @param {Object} cluster Cluster to process.
+     * @param {Boolean} client
+     * @return {Bean} Generated ignite configuration.
+     */
+    static igniteConfiguration(cluster, client) {
+        const cfg = this.igniteConfigurationBean(cluster);
+
+        this.clusterGeneral(cluster, cfg, client);
+        this.clusterAtomics(cluster.atomicConfiguration, cfg);
+        this.clusterBinary(cluster.binaryConfiguration, cfg);
+        this.clusterCacheKeyConfiguration(cluster.cacheKeyConfiguration, cfg);
+        this.clusterCheckpoint(cluster, cluster.caches, cfg);
+        this.clusterCollision(cluster.collision, cfg);
+        this.clusterCommunication(cluster, cfg);
+        this.clusterConnector(cluster.connector, cfg);
+        this.clusterDeployment(cluster, cfg);
+        this.clusterEvents(cluster, cfg);
+        this.clusterFailover(cluster, cfg);
+        this.clusterLoadBalancing(cluster, cfg);
+        this.clusterLogger(cluster.logger, cfg);
+        this.clusterODBC(cluster.odbc, cfg);
+        this.clusterMarshaller(cluster, cfg);
+        this.clusterMetrics(cluster, cfg);
+        this.clusterSwap(cluster, cfg);
+        this.clusterTime(cluster, cfg);
+        this.clusterPools(cluster, cfg);
+        this.clusterTransactions(cluster.transactionConfiguration, cfg);
+        this.clusterSsl(cluster, cfg);
+        this.clusterUserAttributes(cluster, cfg);
+
+        this.clusterCaches(cluster, cluster.caches, cluster.igfss, client, cfg);
+
+        if (!client)
+            this.clusterIgfss(cluster.igfss, cfg);
+
+        return cfg;
+    }
 
-                    break;
-                case 'Oracle':
-                    dsBean = new Bean('oracle.jdbc.pool.OracleDataSource', id, {})
-                        .property('URL', `${id}.jdbc.url`, 'jdbc:oracle:thin:@[host]:[port]:[database]');
+    static dialectClsName(dialect) {
+        return DFLT_DIALECTS[dialect] || 'Unknown database: ' + (dialect || 'Choose JDBC dialect');
+    }
 
-                    break;
-                case 'DB2':
-                    dsBean = new Bean('com.ibm.db2.jcc.DB2DataSource', id, {})
-                        .property('serverName', `${id}.jdbc.server_name`, 'YOUR_DATABASE_SERVER_NAME')
-                        .propertyInt('portNumber', `${id}.jdbc.port_number`, 'YOUR_JDBC_PORT_NUMBER')
-                        .property('databaseName', `${id}.jdbc.database_name`, 'YOUR_DATABASE_NAME')
-                        .propertyInt('driverType', `${id}.jdbc.driver_type`, 'YOUR_JDBC_DRIVER_TYPE');
+    static dataSourceBean(id, dialect) {
+        let dsBean;
+
+        switch (dialect) {
+            case 'Generic':
+                dsBean = new Bean('com.mchange.v2.c3p0.ComboPooledDataSource', id, {})
+                    .property('jdbcUrl', `${id}.jdbc.url`, 'jdbc:your_database');
+
+                break;
+            case 'Oracle':
+                dsBean = new Bean('oracle.jdbc.pool.OracleDataSource', id, {})
+                    .property('URL', `${id}.jdbc.url`, 'jdbc:oracle:thin:@[host]:[port]:[database]');
+
+                break;
+            case 'DB2':
+                dsBean = new Bean('com.ibm.db2.jcc.DB2DataSource', id, {})
+                    .property('serverName', `${id}.jdbc.server_name`, 'YOUR_DATABASE_SERVER_NAME')
+                    .propertyInt('portNumber', `${id}.jdbc.port_number`, 'YOUR_JDBC_PORT_NUMBER')
+                    .property('databaseName', `${id}.jdbc.database_name`, 'YOUR_DATABASE_NAME')
+                    .propertyInt('driverType', `${id}.jdbc.driver_type`, 'YOUR_JDBC_DRIVER_TYPE');
+
+                break;
+            case 'SQLServer':
+                dsBean = new Bean('com.microsoft.sqlserver.jdbc.SQLServerDataSource', id, {})
+                    .property('URL', `${id}.jdbc.url`, 'jdbc:sqlserver://[host]:[port][;databaseName=database]');
+
+                break;
+            case 'MySQL':
+                dsBean = new Bean('com.mysql.jdbc.jdbc2.optional.MysqlDataSource', id, {})
+                    .property('URL', `${id}.jdbc.url`, 'jdbc:mysql://[host]:[port]/[database]');
+
+                break;
+            case 'PostgreSQL':
+                dsBean = new Bean('org.postgresql.ds.PGPoolingDataSource', id, {})
+                    .property('url', `${id}.jdbc.url`, 'jdbc:postgresql://[host]:[port]/[database]');
+
+                break;
+            case 'H2':
+                dsBean = new Bean('org.h2.jdbcx.JdbcDataSource', id, {})
+                    .property('URL', `${id}.jdbc.url`, 'jdbc:h2:tcp://[host]/[database]');
+
+                break;
+            default:
+        }
 
-                    break;
-                case 'SQLServer':
-                    dsBean = new Bean('com.microsoft.sqlserver.jdbc.SQLServerDataSource', id, {})
-                        .property('URL', `${id}.jdbc.url`, 'jdbc:sqlserver://[host]:[port][;databaseName=database]');
+        if (dsBean) {
+            dsBean.property('user', `${id}.jdbc.username`, 'YOUR_USER_NAME')
+                .property('password', `${id}.jdbc.password`, 'YOUR_PASSWORD');
+        }
 
-                    break;
-                case 'MySQL':
-                    dsBean = new Bean('com.mysql.jdbc.jdbc2.optional.MysqlDataSource', id, {})
-                        .property('URL', `${id}.jdbc.url`, 'jdbc:mysql://[host]:[port]/[database]');
+        return dsBean;
+    }
 
-                    break;
-                case 'PostgreSQL':
-                    dsBean = new Bean('org.postgresql.ds.PGPoolingDataSource', id, {})
-                        .property('url', `${id}.jdbc.url`, 'jdbc:postgresql://[host]:[port]/[database]');
+    // Generate general section.
+    static clusterGeneral(cluster, cfg = this.igniteConfigurationBean(cluster), client = false) {
+        if (client)
+            cfg.prop('boolean', 'clientMode', true);
 
-                    break;
-                case 'H2':
-                    dsBean = new Bean('org.h2.jdbcx.JdbcDataSource', id, {})
-                        .property('URL', `${id}.jdbc.url`, 'jdbc:h2:tcp://[host]/[database]');
+        cfg.stringProperty('name', 'gridName')
+            .stringProperty('localHost');
 
-                    break;
-                default:
-            }
-
-            if (dsBean) {
-                dsBean.property('user', `${id}.jdbc.username`, 'YOUR_USER_NAME')
-                    .property('password', `${id}.jdbc.password`, 'YOUR_PASSWORD');
-            }
+        if (_.isNil(cluster.discovery))
+            return cfg;
 
-            return dsBean;
-        }
+        const discovery = new Bean('org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi', 'discovery',
+            cluster.discovery, clusterDflts.discovery);
 
-        // Generate general section.
-        static clusterGeneral(cluster, cfg = this.igniteConfigurationBean(cluster), client = false) {
-            if (client)
-                cfg.prop('boolean', 'clientMode', true);
+        let ipFinder;
 
-            cfg.stringProperty('name', 'gridName')
-                .stringProperty('localHost');
+        switch (discovery.valueOf('kind')) {
+            case 'Vm':
+                ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder',
+                    'ipFinder', cluster.discovery.Vm, clusterDflts.discovery.Vm);
 
-            if (_.isNil(cluster.discovery))
-                return cfg;
+                ipFinder.collectionProperty('addrs', 'addresses', cluster.discovery.Vm.addresses);
 
-            const discovery = new Bean('org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi', 'discovery',
-                cluster.discovery, clusterDflts.discovery);
+                break;
+            case 'Multicast':
+                ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder',
+                    'ipFinder', cluster.discovery.Multicast, clusterDflts.discovery.Multicast);
 
-            let ipFinder;
+                ipFinder.stringProperty('multicastGroup')
+                    .intProperty('multicastPort')
+                    .intProperty('responseWaitTime')
+                    .intProperty('addressRequestAttempts')
+                    .stringProperty('localAddress')
+                    .collectionProperty('addrs', 'addresses', cluster.discovery.Multicast.addresses);
 
-            switch (discovery.valueOf('kind')) {
-                case 'Vm':
-                    ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder',
-                        'ipFinder', cluster.discovery.Vm, clusterDflts.discovery.Vm);
+                break;
+            case 'S3':
+                ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.s3.TcpDiscoveryS3IpFinder',
+                    'ipFinder', cluster.discovery.S3, clusterDflts.discovery.S3);
 
-                    ipFinder.collectionProperty('addrs', 'addresses', cluster.discovery.Vm.addresses);
+                ipFinder.stringProperty('bucketName');
 
-                    break;
-                case 'Multicast':
-                    ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder',
-                        'ipFinder', cluster.discovery.Multicast, clusterDflts.discovery.Multicast);
+                break;
+            case 'Cloud':
+                ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.cloud.TcpDiscoveryCloudIpFinder',
+                    'ipFinder', cluster.discovery.Cloud, clusterDflts.discovery.Cloud);
 
-                    ipFinder.stringProperty('multicastGroup')
-                        .intProperty('multicastPort')
-                        .intProperty('responseWaitTime')
-                        .intProperty('addressRequestAttempts')
-                        .stringProperty('localAddress')
-                        .collectionProperty('addrs', 'addresses', cluster.discovery.Multicast.addresses);
+                ipFinder.stringProperty('credential')
+                    .pathProperty('credentialPath')
+                    .stringProperty('identity')
+                    .stringProperty('provider')
+                    .collectionProperty('regions', 'regions', cluster.discovery.Cloud.regions)
+                    .collectionProperty('zones', 'zones', cluster.discovery.Cloud.zones);
 
-                    break;
-                case 'S3':
-                    ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.s3.TcpDiscoveryS3IpFinder',
-                        'ipFinder', cluster.discovery.S3, clusterDflts.discovery.S3);
+                break;
+            case 'GoogleStorage':
+                ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.gce.TcpDiscoveryGoogleStorageIpFinder',
+                    'ipFinder', cluster.discovery.GoogleStorage, clusterDflts.discovery.GoogleStorage);
 
-                    ipFinder.stringProperty('bucketName');
+                ipFinder.stringProperty('projectName')
+                    .stringProperty('bucketName')
+                    .pathProperty('serviceAccountP12FilePath')
+                    .stringProperty('serviceAccountId');
 
-                    break;
-                case 'Cloud':
-                    ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.cloud.TcpDiscoveryCloudIpFinder',
-                        'ipFinder', cluster.discovery.Cloud, clusterDflts.discovery.Cloud);
+                break;
+            case 'Jdbc':
+                ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinder',
+                    'ipFinder', cluster.discovery.Jdbc, clusterDflts.discovery.Jdbc);
 
-                    ipFinder.stringProperty('credential')
-                        .pathProperty('credentialPath')
-                        .stringProperty('identity')
-                        .stringProperty('provider')
-                        .collectionProperty('regions', 'regions', cluster.discovery.Cloud.regions)
-                        .collectionProperty('zones', 'zones', cluster.discovery.Cloud.zones);
+                ipFinder.intProperty('initSchema');
 
-                    break;
-                case 'GoogleStorage':
-                    ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.gce.TcpDiscoveryGoogleStorageIpFinder',
-                        'ipFinder', cluster.discovery.GoogleStorage, clusterDflts.discovery.GoogleStorage);
+                if (ipFinder.includes('dataSourceBean', 'dialect')) {
+                    const id = ipFinder.valueOf('dataSourceBean');
 
-                    ipFinder.stringProperty('projectName')
-                        .stringProperty('bucketName')
-                        .pathProperty('serviceAccountP12FilePath')
-                        .stringProperty('serviceAccountId');
+                    ipFinder.dataSource(id, 'dataSource', this.dataSourceBean(id, ipFinder.valueOf('dialect')));
+                }
 
-                    break;
-                case 'Jdbc':
-                    ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinder',
-                        'ipFinder', cluster.discovery.Jdbc, clusterDflts.discovery.Jdbc);
+                break;
+            case 'SharedFs':
+                ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.sharedfs.TcpDiscoverySharedFsIpFinder',
+                    'ipFinder', cluster.discovery.SharedFs, clusterDflts.discovery.SharedFs);
+
+                ipFinder.pathProperty('path');
+
+                break;
+            case 'ZooKeeper':
+                const src = cluster.discovery.ZooKeeper;
+                const dflt = clusterDflts.discovery.ZooKeeper;
+
+                ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.zk.TcpDiscoveryZookeeperIpFinder',
+                    'ipFinder', src, dflt);
+
+                ipFinder.emptyBeanProperty('curator')
+                    .stringProperty('zkConnectionString');
+
+                const kind = _.get(src, 'retryPolicy.kind');
+
+                if (kind) {
+                    const policy = src.retryPolicy;
+
+                    let retryPolicyBean;
+
+                    switch (kind) {
+                        case 'ExponentialBackoff':
+                            retryPolicyBean = new Bean('org.apache.curator.retry.ExponentialBackoffRetry', null,
+                                policy.ExponentialBackoff, dflt.ExponentialBackoff)
+                                .intConstructorArgument('baseSleepTimeMs')
+                                .intConstructorArgument('maxRetries')
+                                .intConstructorArgument('maxSleepMs');
+
+                            break;
+                        case 'BoundedExponentialBackoff':
+                            retryPolicyBean = new Bean('org.apache.curator.retry.BoundedExponentialBackoffRetry',
+                                null, policy.BoundedExponentialBackoff, dflt.BoundedExponentialBackoffRetry)
+                                .intConstructorArgument('baseSleepTimeMs')
+                                .intConstructorArgument('maxSleepTimeMs')
+                                .intConstructorArgument('maxRetries');
+
+                            break;
+                        case 'UntilElapsed':
+                            retryPolicyBean = new Bean('org.apache.curator.retry.RetryUntilElapsed', null,
+                                policy.UntilElapsed, dflt.UntilElapsed)
+                                .intConstructorArgument('maxElapsedTimeMs')
+                                .intConstructorArgument('sleepMsBetweenRetries');
+
+                            break;
+
+                        case 'NTimes':
+                            retryPolicyBean = new Bean('org.apache.curator.retry.RetryNTimes', null,
+                                policy.NTimes, dflt.NTimes)
+                                .intConstructorArgument('n')
+                                .intConstructorArgument('sleepMsBetweenRetries');
+
+                            break;
+                        case 'OneTime':
+                            retryPolicyBean = new Bean('org.apache.curator.retry.RetryOneTime', null,
+                                policy.OneTime, dflt.OneTime)
+                                .intConstructorArgument('sleepMsBetweenRetry');
+
+                            break;
+                        case 'Forever':
+                            retryPolicyBean = new Bean('org.apache.curator.retry.RetryForever', null,
+                                policy.Forever, dflt.Forever)
+                                .intConstructorArgument('retryIntervalMs');
+
+                            break;
+                        case 'Custom':
+                            const className = _.get(policy, 'Custom.className');
+
+                            if (_.nonEmpty(className))
+                                retryPolicyBean = new EmptyBean(className);
+
+                            break;
+                        default:
+                            // No-op.
+                    }
 
-                    ipFinder.intProperty('initSchema');
+                    if (retryPolicyBean)
+                        ipFinder.beanProperty('retryPolicy', retryPolicyBean);
+                }
 
-                    if (ipFinder.includes('dataSourceBean', 'dialect')) {
-                        const id = ipFinder.valueOf('dataSourceBean');
+                ipFinder.pathProperty('basePath', '/services')
+                    .stringProperty('serviceName')
+                    .boolProperty('allowDuplicateRegistrations');
 
-                        ipFinder.dataSource(id, 'dataSource', this.dataSourceBean(id, ipFinder.valueOf('dialect')));
-                    }
+                break;
+            default:
+                // No-op.
+        }
 
-                    break;
-                case 'SharedFs':
-                    ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.sharedfs.TcpDiscoverySharedFsIpFinder',
-                        'ipFinder', cluster.discovery.SharedFs, clusterDflts.discovery.SharedFs);
+        if (ipFinder)
+            discovery.beanProperty('ipFinder', ipFinder);
 
-                    ipFinder.pathProperty('path');
+        this.clusterDiscovery(cluster.discovery, cfg, discovery);
 
-                    break;
-                case 'ZooKeeper':
-                    const src = cluster.discovery.ZooKeeper;
-                    const dflt = clusterDflts.discovery.ZooKeeper;
+        return cfg;
+    }
 
-                    ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.zk.TcpDiscoveryZookeeperIpFinder',
-                        'ipFinder', src, dflt);
+    static igfsDataCache(igfs) {
+        return this.cacheConfiguration({
+            name: igfs.name + '-data',
+            cacheMode: 'PARTITIONED',
+            atomicityMode: 'TRANSACTIONAL',
+            writeSynchronizationMode: 'FULL_SYNC',
+            backups: 0,
+            igfsAffinnityGroupSize: igfs.affinnityGroupSize || 512
+        });
+    }
 
-                    ipFinder.emptyBeanProperty('curator')
-                        .stringProperty('zkConnectionString');
+    static igfsMetaCache(igfs) {
+        return this.cacheConfiguration({
+            name: igfs.name + '-meta',
+            cacheMode: 'REPLICATED',
+            atomicityMode: 'TRANSACTIONAL',
+            writeSynchronizationMode: 'FULL_SYNC'
+        });
+    }
 
-                    const kind = _.get(src, 'retryPolicy.kind');
+    static clusterCaches(cluster, caches, igfss, client, cfg = this.igniteConfigurationBean(cluster)) {
+        const ccfgs = _.map(caches, (cache) => this.cacheConfiguration(cache));
 
-                    if (kind) {
-                        const policy = src.retryPolicy;
+        if (!client) {
+            _.forEach(igfss, (igfs) => {
+                ccfgs.push(this.igfsDataCache(igfs));
+                ccfgs.push(this.igfsMetaCache(igfs));
+            });
+        }
 
-                        let retryPolicyBean;
+        cfg.varArgProperty('ccfgs', 'cacheConfiguration', ccfgs, 'org.apache.ignite.configuration.CacheConfiguration');
 
-                        switch (kind) {
-                            case 'ExponentialBackoff':
-                                retryPolicyBean = new Bean('org.apache.curator.retry.ExponentialBackoffRetry', null,
-                                    policy.ExponentialBackoff, dflt.ExponentialBackoff)
-                                    .intConstructorArgument('baseSleepTimeMs')
-                                    .intConstructorArgument('maxRetries')
-                                    .intConstructorArgument('maxSleepMs');
+        return cfg;
+    }
 
-                                break;
-                            case 'BoundedExponentialBackoff':
-                                retryPolicyBean = new Bean('org.apache.curator.retry.BoundedExponentialBackoffRetry',
-                                    null, policy.BoundedExponentialBackoffRetry, dflt.BoundedExponentialBackoffRetry)
-                                    .intConstructorArgument('baseSleepTimeMs')
-                                    .intConstructorArgument('maxSleepTimeMs')
-                                    .intConstructorArgument('maxRetries');
+    // Generate atomics group.
+    static clusterAtomics(atomics, cfg = this.igniteConfigurationBean()) {
+        const acfg = new Bean('org.apache.ignite.configuration.AtomicConfiguration', 'atomicCfg',
+            atomics, clusterDflts.atomics);
 
-                                break;
-                            case 'UntilElapsed':
-                                retryPolicyBean = new Bean('org.apache.curator.retry.RetryUntilElapsed', null,
-                                    policy.UntilElapsed, dflt.UntilElapsed)
-                                    .intConstructorArgument('maxElapsedTimeMs')
-                                    .intConstructorArgument('sleepMsBetweenRetries');
+        acfg.enumProperty('cacheMode')
+            .intProperty('atomicSequenceReserveSize');
 
-                                break;
+        if (acfg.valueOf('cacheMode') === 'PARTITIONED')
+            acfg.intProperty('backups');
 
-                            case 'NTimes':
-                                retryPolicyBean = new Bean('org.apache.curator.retry.RetryNTimes', null,
-                                    policy.NTimes, dflt.NTimes)
-                                    .intConstructorArgument('n')
-                                    .intConstructorArgument('sleepMsBetweenRetries');
+        if (acfg.isEmpty())
+            return cfg;
 
-                                break;
-                            case 'OneTime':
-                                retryPolicyBean = new Bean('org.apache.curator.retry.RetryOneTime', null,
-                                    policy.OneTime, dflt.OneTime)
-                                    .intConstructorArgument('sleepMsBetweenRetry');
+        cfg.beanProperty('atomicConfiguration', acfg);
 
-                                break;
-                            case 'Forever':
-                                retryPolicyBean = new Bean('org.apache.curator.retry.RetryForever', null,
-                                    policy.Forever, dflt.Forever)
-                                    .intConstructorArgument('retryIntervalMs');
+        return cfg;
+    }
 
-                                break;
-                            case 'Custom':
-                                if (_.nonEmpty(policy.Custom.className))
-                                    retryPolicyBean = new EmptyBean(policy.Custom.className);
+    // Generate binary group.
+    static clusterBinary(binary, cfg = this.igniteConfigurationBean()) {
+        const binaryCfg = new Bean('org.apache.ignite.configuration.BinaryConfiguration', 'binaryCfg',
+            binary, clusterDflts.binary);
 
-                                break;
-                            default:
-                                // No-op.
-                        }
+        binaryCfg.emptyBeanProperty('idMapper')
+            .emptyBeanProperty('nameMapper')
+            .emptyBeanProperty('serializer');
 
-                        if (retryPolicyBean)
-                            ipFinder.beanProperty('retryPolicy', retryPolicyBean);
-                    }
+        const typeCfgs = [];
 
-                    ipFinder.pathProperty('basePath', '/services')
-                        .stringProperty('serviceName')
-                        .boolProperty('allowDuplicateRegistrations');
+        _.forEach(binary.typeConfigurations, (type) => {
+            const typeCfg = new Bean('org.apache.ignite.binary.BinaryTypeConfiguration',
+                javaTypes.toJavaName('binaryType', type.typeName), type, clusterDflts.binary.typeConfigurations);
 
-                    break;
-                default:
-                    // No-op.
-            }
+            typeCfg.stringProperty('typeName')
+                .emptyBeanProperty('idMapper')
+                .emptyBeanProperty('nameMapper')
+                .emptyBeanProperty('serializer')
+                .intProperty('enum');
 
-            if (ipFinder)
-                discovery.beanProperty('ipFinder', ipFinder);
+            if (typeCfg.nonEmpty())
+                typeCfgs.push(typeCfg);
+        });
 
-            this.clusterDiscovery(cluster.discovery, cfg, discovery);
+        binaryCfg.collectionProperty('types', 'typeConfigurations', typeCfgs, 'org.apache.ignite.binary.BinaryTypeConfiguration')
+            .boolProperty('compactFooter');
 
+        if (binaryCfg.isEmpty())
             return cfg;
-        }
-
-        static igfsDataCache(igfs) {
-            return this.cacheConfiguration({
-                name: igfs.name + '-data',
-                cacheMode: 'PARTITIONED',
-                atomicityMode: 'TRANSACTIONAL',
-                writeSynchronizationMode: 'FULL_SYNC',
-                backups: 0,
-                igfsAffinnityGroupSize: igfs.affinnityGroupSize || 512
-            });
-        }
 
-        static igfsMetaCache(igfs) {
-            return this.cacheConfiguration({
-                name: igfs.name + '-meta',
-                cacheMode: 'REPLICATED',
-                atomicityMode: 'TRANSACTIONAL',
-                writeSynchronizationMode: 'FULL_SYNC'
-            });
-        }
+        cfg.beanProperty('binaryConfiguration', binaryCfg);
 
-        static clusterCaches(cluster, caches, igfss, client, cfg = this.igniteConfigurationBean(cluster)) {
-            const ccfgs = _.map(caches, (cache) => this.cacheConfiguration(cache));
+        return cfg;
+    }
 
-            if (!client) {
-                _.forEach(igfss, (igfs) => {
-                    ccfgs.push(this.igfsDataCache(igfs));
-                    ccfgs.push(this.igfsMetaCache(igfs));
-                });
+    // Generate cache key configurations.
+    static clusterCacheKeyConfiguration(keyCfgs, cfg = this.igniteConfigurationBean()) {
+        const items = _.reduce(keyCfgs, (acc, keyCfg) => {
+            if (keyCfg.typeName && keyCfg.affinityKeyFieldName) {
+                acc.push(new Bean('org.apache.ignite.cache.CacheKeyConfiguration', null, keyCfg)
+                    .stringConstructorArgument('typeName')
+                    .stringConstructorArgument('affinityKeyFieldName'));
             }
 
-            cfg.varArgProperty('ccfgs', 'cacheConfiguration', ccfgs, 'org.apache.ignite.configuration.CacheConfiguration');
+            return acc;
+        }, []);
 
+        if (_.isEmpty(items))
             return cfg;
-        }
 
-        // Generate atomics group.
-        static clusterAtomics(atomics, cfg = this.igniteConfigurationBean()) {
-            const acfg = new Bean('org.apache.ignite.configuration.AtomicConfiguration', 'atomicCfg',
-                atomics, clusterDflts.atomics);
+        cfg.arrayProperty('cacheKeyConfiguration', 'cacheKeyConfiguration', items,
+            'org.apache.ignite.cache.CacheKeyConfiguration');
 
-            acfg.enumProperty('cacheMode')
-                .intProperty('atomicSequenceReserveSize');
+        return cfg;
+    }
 
-            if (acfg.valueOf('cacheMode') === 'PARTITIONED')
-                acfg.intProperty('backups');
+    // Generate checkpoint configurations.
+    static clusterCheckpoint(cluster, caches, cfg = this.igniteConfigurationBean()) {
+        const cfgs = _.filter(_.map(cluster.checkpointSpi, (spi) => {
+            switch (_.get(spi, 'kind')) {
+                case 'FS':
+                    const fsBean = new Bean('org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointSpi',
+                        'checkpointSpiFs', spi.FS);
 
-            if (acfg.isEmpty())
-                return cfg;
+                    fsBean.collectionProperty('directoryPaths', 'directoryPaths', _.get(spi, 'FS.directoryPaths'))
+                        .emptyBeanProperty('checkpointListener');
 
-            cfg.beanProperty('atomicConfiguration', acfg);
+                    return fsBean;
 
-            return cfg;
-        }
+                case 'Cache':
+                    const cacheBean = new Bean('org.apache.ignite.spi.checkpoint.cache.CacheCheckpointSpi',
+                        'checkpointSpiCache', spi.Cache);
 
-        // Generate binary group.
-        static clusterBinary(binary, cfg = this.igniteConfigurationBean()) {
-            const binaryCfg = new Bean('org.apache.ignite.configuration.BinaryConfiguration', 'binaryCfg',
-                binary, clusterDflts.binary);
+                    const curCache = _.get(spi, 'Cache.cache');
 
-            binaryCfg.emptyBeanProperty('idMapper')
-                .emptyBeanProperty('nameMapper')
-                .emptyBeanProperty('serializer');
+                    const cache = _.find(caches, (c) => curCache && (c._id === curCache || _.get(c, 'cache._id') === curCache));
 
-            const typeCfgs = [];
+                    if (cache)
+                        cacheBean.prop('java.lang.String', 'cacheName', cache.name || cache.cache.name);
 
-            _.forEach(binary.typeConfigurations, (type) => {
-                const typeCfg = new Bean('org.apache.ignite.binary.BinaryTypeConfiguration',
-                    JavaTypes.toJavaName('binaryType', type.typeName), type, clusterDflts.binary.typeConfigurations);
+                    cacheBean.stringProperty('cacheName')
+                        .emptyBeanProperty('checkpointListener');
 
-                typeCfg.stringProperty('typeName')
-                    .emptyBeanProperty('idMapper')
-                    .emptyBeanProperty('nameMapper')
-                    .emptyBeanProperty('serializer')
-                    .intProperty('enum');
+                    return cacheBean;
 
-                if (typeCfg.nonEmpty())
-                    typeCfgs.push(typeCfg);
-            });
+                case 'S3':
+                    const s3Bean = new Bean('org.apache.ignite.spi.checkpoint.s3.S3CheckpointSpi',
+                        'checkpointSpiS3', spi.S3, clusterDflts.checkpointSpi.S3);
 
-            binaryCfg.collectionProperty('types', 'typeConfigurations', typeCfgs, 'org.apache.ignite.binary.BinaryTypeConfiguration')
-                .boolProperty('compactFooter');
+                    let credentialsBean = null;
 
-            if (binaryCfg.isEmpty())
-                return cfg;
+                    switch (_.get(spi.S3, 'awsCredentials.kind')) {
+                        case 'Basic':
+                            credentialsBean = new Bean('com.amazonaws.auth.BasicAWSCredentials', 'awsCredentials', {});
 
-            cfg.beanProperty('binaryConfiguration', binaryCfg);
+                            credentialsBean.propertyConstructorArgument('checkpoint.s3.credentials.accessKey', 'YOUR_S3_ACCESS_KEY')
+                                .propertyConstructorArgument('checkpoint.s3.credentials.secretKey', 'YOUR_S3_SECRET_KEY');
 
-            return cfg;
-        }
+                            break;
 
-        // Generate cache key configurations.
-        static clusterCacheKeyConfiguration(keyCfgs, cfg = this.igniteConfigurationBean()) {
-            const items = _.reduce(keyCfgs, (acc, keyCfg) => {
-                if (keyCfg.typeName && keyCfg.affinityKeyFieldName) {
-                    acc.push(new Bean('org.apache.ignite.cache.CacheKeyConfiguration', null, keyCfg)
-                        .stringConstructorArgument('typeName')
-                        .stringConstructorArgument('affinityKeyFieldName'));
-                }
+                        case 'Properties':
+                            credentialsBean = new Bean('com.amazonaws.auth.PropertiesCredentials', 'awsCredentials', {});
 
-                return acc;
-            }, []);
+                            const fileBean = new Bean('java.io.File', '', spi.S3.awsCredentials.Properties)
+                                .pathConstructorArgument('path');
 
-            if (_.isEmpty(items))
-                return cfg;
+                            if (fileBean.nonEmpty())
+                                credentialsBean.beanConstructorArgument('file', fileBean);
 
-            cfg.arrayProperty('cacheKeyConfiguration', 'cacheKeyConfiguration', items,
-                'org.apache.ignite.cache.CacheKeyConfiguration');
+                            break;
 
-            return cfg;
-        }
+                        case 'Anonymous':
+                            credentialsBean = new Bean('com.amazonaws.auth.AnonymousAWSCredentials', 'awsCredentials', {});
 
-        // Generate checkpoint configurations.
-        static clusterCheckpoint(cluster, caches, cfg = this.igniteConfigurationBean()) {
-            const cfgs = _.filter(_.map(cluster.checkpointSpi, (spi) => {
-                switch (_.get(spi, 'kind')) {
-                    case 'FS':
-                        const fsBean = new Bean('org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointSpi',
-                            'checkpointSpiFs', spi.FS);
+                            break;
 
-                        fsBean.collectionProperty('directoryPaths', 'directoryPaths', _.get(spi, 'FS.directoryPaths'))
-                            .emptyBeanProperty('checkpointListener');
+                        case 'BasicSession':
+                            credentialsBean = new Bean('com.amazonaws.auth.BasicSessionCredentials', 'awsCredentials', {});
 
-                        return fsBean;
+                            // TODO 2054 Arguments in one line is very long string.
+                            credentialsBean.propertyConstructorArgument('checkpoint.s3.credentials.accessKey')
+                                .propertyConstructorArgument('checkpoint.s3.credentials.secretKey')
+                                .propertyConstructorArgument('checkpoint.s3.credentials.sessionToken');
 
-                    case 'Cache':
-                        const cacheBean = new Bean('org.apache.ignite.spi.checkpoint.cache.CacheCheckpointSpi',
-                            'checkpointSpiCache', spi.Cache);
+                            break;
 
-                        const curCache = _.get(spi, 'Cache.cache');
+                        case 'Custom':
+                            const className = _.get(spi.S3.awsCredentials, 'Custom.className');
 
-                        const cache = _.find(caches, (c) => curCache && (c._id === curCache || _.get(c, 'cache._id') === curCache));
+                            if (className)
+                                credentialsBean = new Bean(className, 'awsCredentials', {});
 
-                        if (cache)
-                            cacheBean.prop('java.lang.String', 'cacheName', cache.name || cache.cache.name);
+                            break;
 
-                        cacheBean.stringProperty('cacheName')
-                            .emptyBeanProperty('checkpointListener');
+                        default:
+                            break;
+                    }
 
-                        return cacheBean;
+                    if (credentialsBean)
+                        s3Bean.beanProperty('awsCredentials', credentialsBean);
 
-                    case 'S3':
-                        const s3Bean = new Bean('org.apache.ignite.spi.checkpoint.s3.S3CheckpointSpi',
-                            'checkpointSpiS3', spi.S3, clusterDflts.checkpointSpi.S3);
+                    s3Bean.stringProperty('bucketNameSuffix');
 
-                        let credentialsBean = null;
+                    const clientBean = new Bean('com.amazonaws.ClientConfiguration', 'clientCfg', spi.S3.clientConfiguration,
+                        clusterDflts.checkpointSpi.S3.clientConfiguration);
 
-                        switch (_.get(spi.S3, 'awsCredentials.kind')) {
-                            case 'Basic':
-                                credentialsBean = new Bean('com.amazonaws.auth.BasicAWSCredentials', 'awsCredentials', {});
+                    clientBean.enumProperty('protocol')
+                        .intProperty('maxConnections')
+                        .stringProperty('userAgent');
 
-                                credentialsBean.propertyConstructorArgument('checkpoint.s3.credentials.accessKey', 'YOUR_S3_ACCESS_KEY')
-                                    .propertyConstructorArgument('checkpoint.s3.credentials.secretKey', 'YOUR_S3_SECRET_KEY');
+                    const locAddr = new Bean('java.net.InetAddress', '', spi.S3.clientConfiguration)
+                        .factoryMethod('getByName')
+                        .stringConstructorArgument('localAddress');
 
-                                break;
+                    if (locAddr.nonEmpty())
+                        clientBean.beanProperty('localAddress', locAddr);
 
-                            case 'Properties':
-                                credentialsBean = new Bean('com.amazonaws.auth.PropertiesCredentials', 'awsCredentials', {});
+                    clientBean.stringProperty('proxyHost')
+                        .intProperty('proxyPort')
+                        .stringProperty('proxyUsername');
 
-                                const fileBean = new Bean('java.io.File', '', spi.S3.awsCredentials.Properties)
-                                    .pathConstructorArgument('path');
+                    const userName = clientBean.valueOf('proxyUsername');
 
-                                if (fileBean.nonEmpty())
-                                    credentialsBean.beanConstructorArgument('file', fileBean);
+                    if (userName)
+                        clientBean.property('proxyPassword', `checkpoint.s3.proxy.${userName}.password`);
 
-                                break;
+                    clientBean.stringProperty('proxyDomain')
+                        .stringProperty('proxyWorkstation');
 
-                            case 'Anonymous':
-                                credentialsBean = new Bean('com.amazonaws.auth.AnonymousAWSCredentials', 'awsCredentials', {});
+                    const retryPolicy = spi.S3.clientConfiguration.retryPolicy;
 
-                                break;
+                    if (retryPolicy) {
+                        const kind = retryPolicy.kind;
 
-                            case 'BasicSession':
-                                credentialsBean = new Bean('com.amazonaws.auth.BasicSessionCredentials', 'awsCredentials', {});
+                        const policy = retryPolicy[kind];
 
-                                // TODO 2054 Arguments in one line is very long string.
-                                credentialsBean.propertyConstructorArgument('checkpoint.s3.credentials.accessKey')
-                                    .propertyConstructorArgument('checkpoint.s3.credentials.secretKey')
-                                    .propertyConstructorArgument('checkpoint.s3.credentials.sessionToken');
+                        let retryBean;
+
+                        switch (kind) {
+                            case 'Default':
+                                retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', {
+                                    retryCondition: 'DEFAULT_RETRY_CONDITION',
+                                    backoffStrategy: 'DEFAULT_BACKOFF_STRATEGY',
+                                    maxErrorRetry: 'DEFAULT_MAX_ERROR_RETRY',
+                                    honorMaxErrorRetryInClientConfig: true
+                                }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy);
+
+                                retryBean.constantConstructorArgument('retryCondition')
+                                    .constantConstructorArgument('backoffStrategy')
+                                    .constantConstructorArgument('maxErrorRetry')
+                                    .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
 
                                 break;
 
-                            case 'Custom':
-                                const className = _.get(spi.S3.awsCredentials, 'Custom.className');
+                            case 'DefaultMaxRetries':
+                                retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', {
+                                    retryCondition: 'DEFAULT_RETRY_CONDITION',
+                                    backoffStrategy: 'DEFAULT_BACKOFF_STRATEGY',
+                                    maxErrorRetry: _.get(policy, 'maxErrorRetry') || -1,
+                                    honorMaxErrorRetryInClientConfig: false
+                                }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy);
 
-                                credentialsBean = new Bean(className, 'awsCredentials', {});
+                                retryBean.constantConstructorArgument('retryCondition')
+                                    .constantConstructorArgument('backoffStrategy')
+                                    .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry'))
+                                    .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
 
                                 break;
 
-                            default:
-                                break;
-                        }
+                            case 'DynamoDB':
+                                retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', {
+                                    retryCondition: 'DEFAULT_RETRY_CONDITION',
+                                    backoffStrategy: 'DYNAMODB_DEFAULT_BACKOFF_STRATEGY',
+                                    maxErrorRetry: 'DYNAMODB_DEFAULT_MAX_ERROR_RETRY',
+                                    honorMaxErrorRetryInClientConfig: true
+                                }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy);
 
-                        if (credentialsBean)
-                            s3Bean.beanProperty('awsCredentials', credentialsBean);
+                                retryBean.constantConstructorArgument('retryCondition')
+                                    .constantConstructorArgument('backoffStrategy')
+                                    .constantConstructorArgument('maxErrorRetry')
+                                    .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
 
-                        s3Bean.stringProperty('bucketNameSuffix');
+                                break;
 
-                        const clientBean = new Bean('com.amazonaws.ClientConfiguration', 'clientCfg', spi.S3.clientConfiguration,
-                            clusterDflts.checkpointSpi.S3.clientConfiguration);
+                            case 'DynamoDBMaxRetries':
+                                retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', {
+                                    retryCondition: 'DEFAULT_RETRY_CONDITION',
+                                    backoffStrategy: 'DYNAMODB_DEFAULT_BACKOFF_STRATEGY',
+                                    maxErrorRetry: _.get(policy, 'maxErrorRetry') || -1,
+                                    honorMaxErrorRetryInClientConfig: false
+                                }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy);
 
-                        clientBean.enumProperty('protocol')
-                            .intProperty('maxConnections')
-                            .stringProperty('userAgent');
+                                retryBean.constantConstructorArgument('retryCondition')
+                                    .constantConstructorArgument('backoffStrategy')
+                                    .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry'))
+                                    .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
 
-                        const locAddr = new Bean('java.net.InetAddress', '', spi.S3.clientConfiguration)
-                            .factoryMethod('getByName')
-                            .stringConstructorArgument('localAddress');
+                                break;
 
-                        if (locAddr.nonEmpty())
-                            clientBean.beanProperty('localAddress', locAddr);
+                            case 'Custom':
+                                retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', policy);
 
-                        clientBean.stringProperty('proxyHost')
-                            .intProperty('proxyPort')
-                            .stringProperty('proxyUsername');
+                                retryBean.beanConstructorArgument('retryCondition', retryBean.valueOf('retryCondition') ? new EmptyBean(retryBean.valueOf('retryCondition')) : null)
+                                    .beanConstructorArgument('backoffStrategy', retryBean.valueOf('backoffStrategy') ? new EmptyBean(retryBean.valueOf('backoffStrategy')) : null)
+                                    .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry'))
+                                    .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
 
-                        const userName = clientBean.valueOf('proxyUsername');
+                                break;
 
-                        if (userName)
-                            clientBean.property('proxyPassword', `checkpoint.s3.proxy.${userName}.password`);
+                            default:
+                                break;
+                        }
 
-                        clientBean.stringProperty('proxyDomain')
-                            .stringProperty('proxyWorkstation');
+                        if (retryBean)
+                            clientBean.beanProperty('retryPolicy', retryBean);
+                    }
 
-                        const retryPolicy = spi.S3.clientConfiguration.retryPolicy;
+                    clientBean.intProperty('maxErrorRetry')
+                        .intProperty('socketTimeout')
+                        .intProperty('connectionTimeout')
+                        .intProperty('requestTimeout')
+                        .intProperty('socketSendBufferSizeHints')
+                        .stringProperty('signerOverride')
+                        .intProperty('connectionTTL')
+                        .intProperty('connectionMaxIdleMillis')
+                        .emptyBeanProperty('dnsResolver')
+                        .intProperty('responseMetadataCacheSize')
+                        .emptyBeanProperty('secureRandom')
+                        .boolProperty('useReaper')
+                        .boolProperty('useGzip')
+                        .boolProperty('preemptiveBasicProxyAuth')
+                        .boolProperty('useTcpKeepAlive');
+
+                    if (clientBean.nonEmpty())
+                        s3Bean.beanProperty('clientConfiguration', clientBean);
+
+                    s3Bean.emptyBeanProperty('checkpointListener');
+
+                    return s3Bean;
+
+                case 'JDBC':
+                    const jdbcBean = new Bean('org.apache.ignite.spi.checkpoint.jdbc.JdbcCheckpointSpi',
+                        'checkpointSpiJdbc', spi.JDBC, clusterDflts.checkpointSpi.JDBC);
+
+                    const id = jdbcBean.valueOf('dataSourceBean');
+                    const dialect = _.get(spi.JDBC, 'dialect');
+
+                    jdbcBean.dataSource(id, 'dataSource', this.dataSourceBean(id, dialect));
+
+                    if (!_.isEmpty(jdbcBean.valueOf('user'))) {
+                        jdbcBean.stringProperty('user')
+                            .property('pwd', `checkpoint.${jdbcBean.valueOf('dataSourceBean')}.${jdbcBean.valueOf('user')}.jdbc.password`, 'YOUR_PASSWORD');
+                    }
 
-                        if (retryPolicy) {
-                            const kind = retryPolicy.kind;
+                    jdbcBean.stringProperty('checkpointTableName')
+                        .stringProperty('keyFieldName')
+                        .stringProperty('keyFieldType')
+                        .stringProperty('valueFieldName')
+                        .stringProperty('valueFieldType')
+                        .stringProperty('expireDateFieldName')
+                        .stringProperty('expireDateFieldType')
+                        .intProperty('numberOfRetries')
+                        .emptyBeanProperty('checkpointListener');
 
-                            const policy = retryPolicy[kind];
+                    return jdbcBean;
 
-                            let retryBean;
+                case 'Custom':
+                    const clsName = _.get(spi, 'Custom.className');
 
-                            switch (kind) {
-                                case 'Default':
-                                    retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', {
-                                        retryCondition: 'DEFAULT_RETRY_CONDITION',
-                                        backoffStrategy: 'DEFAULT_BACKOFF_STRATEGY',
-                                        maxErrorRetry: 'DEFAULT_MAX_ERROR_RETRY',
-                                        honorMaxErrorRetryInClientConfig: true
-                                    }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy);
+                    if (clsName)
+                        return new Bean(clsName, 'checkpointSpiCustom', spi.Cache);
 
-                                    retryBean.constantConstructorArgument('retryCondition')
-                                        .constantConstructorArgument('backoffStrategy')
-                                        .constantConstructorArgument('maxErrorRetry')
-                                        .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
+                    return null;
 
-                                    break;
+                default:
+                    return null;
+            }
+        }), (checkpointBean) => _.nonNil(checkpointBean));
 
-                                case 'DefaultMaxRetries':
-                                    retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', {
-                                        retryCondition: 'DEFAULT_RETRY_CONDITION',
-                                        backoffStrategy: 'DEFAULT_BACKOFF_STRATEGY',
-                                        maxErrorRetry: _.get(policy, 'maxErrorRetry') || -1,
-                                        honorMaxErrorRetryInClientConfig: false
-                                    }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy);
+        cfg.arrayProperty('checkpointSpi', 'checkpointSpi', cfgs, 'org.apache.ignite.spi.checkpoint.CheckpointSpi');
 
-                                    retryBean.constantConstructorArgument('retryCondition')
-                                        .constantConstructorArgument('backoffStrategy')
-                                        .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry'))
-                                        .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
+        return cfg;
+    }
 
-                                    break;
+    // Generate collision group.
+    static clusterCollision(collision, cfg = this.igniteConfigurationBean()) {
+        let colSpi;
+
+        switch (_.get(collision, 'kind')) {
+            case 'JobStealing':
+                colSpi = new Bean('org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi',
+                    'colSpi', collision.JobStealing, clusterDflts.collision.JobStealing);
+
+                colSpi.intProperty('activeJobsThreshold')
+                    .intProperty('waitJobsThreshold')
+                    .intProperty('messageExpireTime')
+                    .intProperty('maximumStealingAttempts')
+                    .boolProperty('stealingEnabled')
+                    .emptyBeanProperty('externalCollisionListener')
+                    .mapProperty('stealingAttrs', 'stealingAttributes');
+
+                break;
+            case 'FifoQueue':
+                colSpi = new Bean('org.apache.ignite.spi.collision.fifoqueue.FifoQueueCollisionSpi',
+                    'colSpi', collision.FifoQueue, clusterDflts.collision.FifoQueue);
+
+                colSpi.intProperty('parallelJobsNumber')
+                    .intProperty('waitingJobsNumber');
+
+                break;
+            case 'PriorityQueue':
+                colSpi = new Bean('org.apache.ignite.spi.collision.priorityqueue.PriorityQueueCollisionSpi',
+                    'colSpi', collision.PriorityQueue, clusterDflts.collision.PriorityQueue);
+
+                colSpi.intProperty('parallelJobsNumber')
+                    .intProperty('waitingJobsNumber')
+                    .intProperty('priorityAttributeKey')
+                    .intProperty('jobPriorityAttributeKey')
+                    .intProperty('defaultPriority')
+                    .intProperty('starvationIncrement')
+                    .boolProperty('starvationPreventionEnabled');
+
+                break;
+            case 'Custom':
+                if (_.nonNil(_.get(collision, 'Custom.class')))
+                    colSpi = new EmptyBean(collision.Custom.class);
+
+                break;
+            default:
+                return cfg;
+        }
 
-                                case 'DynamoDB':
-                                    retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', {
-                                        retryCondition: 'DEFAULT_RETRY_CONDITION',
-                                        backoffStrategy: 'DYNAMODB_DEFAULT_BACKOFF_STRATEGY',
-                                        maxErrorRetry: 'DYNAMODB_DEFAULT_MAX_ERROR_RETRY',
-                                        honorMaxErrorRetryInClientConfig: true
-                                    }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy);
+        if (_.nonNil(colSpi))
+            cfg.beanProperty('collisionSpi', colSpi);
 
-                                    retryBean.constantConstructorArgument('retryCondition')
-                                        .constantConstructorArgument('backoffStrategy')
-                                        .constantConstructorArgument('maxErrorRetry')
-                                        .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
+        return cfg;
+    }
 
-                                    break;
+    // Generate communication group.
+    static clusterCommunication(cluster, cfg = this.igniteConfigurationBean(cluster)) {
+        const commSpi = new Bean('org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi', 'communicationSpi',
+            cluster.communication, clusterDflts.communication);
+
+        commSpi.emptyBeanProperty('listener')
+            .stringProperty('localAddress')
+            .intProperty('localPort')
+            .intProperty('localPortRange')
+            .intProperty('sharedMemoryPort')
+            .intProperty('directBuffer')
+            .intProperty('directSendBuffer')
+            .intProperty('idleConnectionTimeout')
+            .intProperty('connectTimeout')
+            .intProperty('maxConnectTimeout')
+            .intProperty('reconnectCount')
+            .intProperty('socketSendBuffer')
+            .intProperty('socketReceiveBuffer')
+            .intProperty('messageQueueLimit')
+            .intProperty('slowClientQueueLimit')
+            .intProperty('tcpNoDelay')
+            .intProperty('ackSendThreshold')
+            .intProperty('unacknowledgedMessagesBufferSize')
+            .intProperty('socketWriteTimeout')
+            .intProperty('selectorsCount')
+            .emptyBeanProperty('addressResolver');
+
+        if (commSpi.nonEmpty())
+            cfg.beanProperty('communicationSpi', commSpi);
+
+        cfg.intProperty('networkTimeout')
+            .intProperty('networkSendRetryDelay')
+            .intProperty('networkSendRetryCount')
+            .intProperty('discoveryStartupDelay');
+
+        return cfg;
+    }
 
-                                case 'DynamoDBMaxRetries':
-                                    retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', {
-                                        retryCondition: 'DEFAULT_RETRY_CONDITION',
-                                        backoffStrategy: 'DYNAMODB_DEFAULT_BACKOFF_STRATEGY',
-                                        maxErrorRetry: _.get(policy, 'maxErrorRetry') || -1,
-                                        honorMaxErrorRetryInClientConfig: false
-                                    }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy);
+    // Generate REST access configuration.
+    static clusterConnector(connector, cfg = this.igniteConfigurationBean()) {
+        const connCfg = new Bean('org.apache.ignite.configuration.ConnectorConfiguration',
+            'connectorConfiguration', connector, clusterDflts.connector);
+
+        if (connCfg.valueOf('enabled')) {
+            connCfg.pathProperty('jettyPath')
+                .stringProperty('host')
+                .intProperty('port')
+                .intProperty('portRange')
+                .intProperty('idleTimeout')
+                .intProperty('idleQueryCursorTimeout')
+                .intProperty('idleQueryCursorCheckFrequency')
+                .intProperty('receiveBufferSize')
+                .intProperty('sendBufferSize')
+                .intProperty('sendQueueLimit')
+                .intProperty('directBuffer')
+                .intProperty('noDelay')
+                .intProperty('selectorCount')
+                .intProperty('threadPoolSize')
+                .emptyBeanProperty('messageInterceptor')
+                .stringProperty('secretKey');
+
+            if (connCfg.valueOf('sslEnabled')) {
+                connCfg.intProperty('sslClientAuth')
+                    .emptyBeanProperty('sslFactory');
+            }
 
-                                    retryBean.constantConstructorArgument('retryCondition')
-                                        .constantConstructorArgument('backoffStrategy')
-                                        .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry'))
-                                        .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
+            if (connCfg.nonEmpty())
+                cfg.beanProperty('connectorConfiguration', connCfg);
+        }
 
-                                    break;
+        return cfg;
+    }
 
-                                case 'Custom':
-                                    retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', policy);
+    // Generate deployment group.
+    static clusterDeployment(cluster, cfg = this.igniteConfigurationBean(cluster)) {
+        cfg.enumProperty('deploymentMode')
+            .boolProperty('peerClassLoadingEnabled');
 
-                                    retryBean.beanConstructorArgument('retryCondition', retryBean.valueOf('retryCondition') ? new EmptyBean(retryBean.valueOf('retryCondition')) : null)
-                                        .beanConstructorArgument('backoffStrategy', retryBean.valueOf('backoffStrategy') ? new EmptyBean(retryBean.valueOf('backoffStrategy')) : null)
-                                        .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry'))
-                                        .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
+        if (cfg.valueOf('peerClassLoadingEnabled')) {
+            cfg.intProperty('peerClassLoadingMissedResourcesCacheSize')
+                .intProperty('peerClassLoadingThreadPoolSize')
+                .varArgProperty('p2pLocClsPathExcl', 'peerClassLoadingLocalClassPathExclude',
+                   cluster.peerClassLoadingLocalClassPathExclude);
+        }
 
-                                    break;
+        let deploymentBean = null;
 
-                                default:
-                                    break;
-                            }
+        switch (_.get(cluster, 'deploymentSpi.kind')) {
+            case 'URI':
+                const uriDeployment = cluster.deploymentSpi.URI;
 
-                            if (retryBean)
-                                clientBean.beanProperty('retryPolicy', retryBean);
-                        }
+                deploymentBean = new Bean('org.apache.ignite.spi.deployment.uri.UriDeploymentSpi', 'deploymentSpi', uriDeployment);
 
-                        clientBean.intProperty('maxErrorRetry')
-                            .intProperty('socketTimeout')
-                            .intProperty('connectionTimeout')
-                            .intProperty('requestTimeout')
-                            .intProperty('socketSendBufferSizeHints')
-                            .stringProperty('signerOverride')
-                            .intProperty('connectionTTL')
-                            .intProperty('connectionMaxIdleMillis')
-                            .emptyBeanProperty('dnsResolver')
-                            .intProperty('responseMetadataCacheSize')
-                            .emptyBeanProperty('secureRandom')
-                            .boolProperty('useReaper')
-                            .boolProperty('useGzip')
-                            .boolProperty('preemptiveBasicProxyAuth')
-                            .boolProperty('useTcpKeepAlive');
-
-                        if (clientBean.nonEmpty())
-                            s3Bean.beanProperty('clientConfiguration', clientBean);
-
-                        s3Bean.emptyBeanProperty('checkpointListener');
-
-                        return s3Bean;
-
-                    case 'JDBC':
-                        const jdbcBean = new Bean('org.apache.ignite.spi.checkpoint.jdbc.JdbcCheckpointSpi',
-                            'checkpointSpiJdbc', spi.JDBC, clusterDflts.checkpointSpi.JDBC);
-
-                        const id = jdbcBean.valueOf('dataSourceBean');
-                        const dialect = _.get(spi.JDBC, 'dialect');
-
-                        jdbcBean.dataSource(id, 'dataSource', this.dataSourceBean(id, dialect));
-
-                        if (!_.isEmpty(jdbcBean.valueOf('user'))) {
-                            jdbcBean.stringProperty('user')
-                                .property('pwd', `checkpoint.${jdbcBean.valueOf('dataSourceBean')}.${jdbcBean.valueOf('user')}.jdbc.password`, 'YOUR_PASSWORD');
-                        }
+                const scanners = _.map(uriDeployment.scanners, (scanner) => new EmptyBean(scanner));
 
-                        jdbcBean.stringProperty('checkpointTableName')
-                            .stringProperty('keyFieldName')
-                            .stringProperty('keyFieldType')
-                            .stringProperty('valueFieldName')
-                            .stringProperty('valueFieldType')
-                            .stringProperty('expireDateFieldName')
-                            .stringProperty('expireDateFieldType')
-                            .intProperty('numberOfRetries')
-                            .emptyBeanProperty('checkpointListener');
+                deploymentBean.collectionProperty('uriList', 'uriList', uriDeployment.uriList)
+                    .stringProperty('temporaryDirectoryPath')
+                    .varArgProperty('scanners', 'scanners', scanners,
+                        'org.apache.ignite.spi.deployment.uri.scanners.UriDeploymentScanner')
+                    .emptyBeanProperty('listener')
+                    .boolProperty('checkMd5')
+                    .boolProperty('encodeUri');
 
-                        return jdbcBean;
+                cfg.beanProperty('deploymentSpi', deploymentBean);
 
-                    case 'Custom':
-                        const clsName = _.get(spi, 'Custom.className');
+                break;
 
-                        if (clsName)
-                            return new Bean(clsName, 'checkpointSpiCustom', spi.Cache);
+            case 'Local':
+                deploymentBean = new Bean('org.apache.ignite.spi.deployment.local.LocalDeploymentSpi', 'deploymentSpi', cluster.deploymentSpi.Local);
 
-                        return null;
+                deploymentBean.emptyBeanProperty('listener');
 
-                    default:
-                        return null;
-                }
-            }), (checkpointBean) => _.nonNil(checkpointBean));
+                cfg.beanProperty('deploymentSpi', deploymentBean);
 
-            cfg.arrayProperty('checkpointSpi', 'checkpointSpi', cfgs, 'org.apache.ignite.spi.checkpoint.CheckpointSpi');
+                break;
 
-            return cfg;
-        }
+            case 'Custom':
+                cfg.emptyBeanProperty('deploymentSpi.Custom.className');
 
-        // Generate collision group.
-        static clusterCollision(collision, cfg = this.igniteConfigurationBean()) {
-            let colSpi;
+                break;
 
-            switch (_.get(collision, 'kind')) {
-                case 'JobStealing':
-                    colSpi = new Bean('org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi',
-                        'colSpi', collision.JobStealing, clusterDflts.collision.JobStealing);
+            default:
+                // No-op.
+        }
 
-                    colSpi.intProperty('activeJobsThreshold')
-                        .intProperty('waitJobsThreshold')
-                        .intProperty('messageExpireTime')
-                        .intProperty('maximumStealingAttempts')
-                        .boolProperty('stealingEnabled')
-                        .emptyBeanProperty('externalCollisionListener')
-                        .mapProperty('stealingAttrs', 'stealingAttributes');
+        return cfg;
+    }
 
-                    break;
-                case 'FifoQueue':
-                    colSpi = new Bean('org.apache.ignite.spi.collision.fifoqueue.FifoQueueCollisionSpi',
-                        'colSpi', collision.FifoQueue, clusterDflts.collision.FifoQueue);
+    // Generate discovery group.
+    static clusterDiscovery(discovery, cfg = this.igniteConfigurationBean(), discoSpi = this.discoveryConfigurationBean(discovery)) {
+        discoSpi.stringProperty('localAddress')
+            .intProperty('localPort')
+            .intProperty('localPortRange')
+            .emptyBeanProperty('addressResolver')
+            .intProperty('socketTimeout')
+            .intProperty('ackTimeout')
+            .intProperty('maxAckTimeout')
+            .intProperty('networkTimeout')
+            .intProperty('joinTimeout')
+            .intProperty('threadPriority')
+            .intProperty('heartbeatFrequency')
+            .intProperty('maxMissedHeartbeats')
+            .intProperty('maxMissedClientHeartbeats')
+            .intProperty('topHistorySize')
+            .emptyBeanProperty('listener')
+            .emptyBeanProperty('dataExchange')
+            .emptyBeanProperty('metricsProvider')
+            .intProperty('reconnectCount')
+            .intProperty('statisticsPrintFrequency')
+            .intProperty('ipFinderCleanFrequency')
+            .emptyBeanProperty('authenticator')
+            .intProperty('forceServerMode')
+            .intProperty('clientReconnectDisabled');
+
+        if (discoSpi.nonEmpty())
+            cfg.beanProperty('discoverySpi', discoSpi);
+
+        return discoSpi;
+    }
 
-                    colSpi.intProperty('parallelJobsNumber')
-                        .intProperty('waitingJobsNumber');
+    // Generate events group.
+    static clusterEvents(cluster, cfg = this.igniteConfigurationBean(cluster)) {
+        const eventStorage = cluster.eventStorage;
 
-                    break;
-                case 'PriorityQueue':
-                    colSpi = new Bean('org.apache.ignite.spi.collision.priorityqueue.PriorityQueueCollisionSpi',
-                        'colSpi', collision.PriorityQueue, clusterDflts.collision.PriorityQueue);
-
-                    colSpi.intProperty('parallelJobsNumber')
-                        .intProperty('waitingJobsNumber')
-                        .intProperty('priorityAttributeKey')
-                        .intProperty('jobPriorityAttributeKey')
-                        .intProperty('defaultPriority')
-                        .intProperty('starvationIncrement')
-                        .boolProperty('starvationPreventionEnabled');
+        let eventStorageBean = null;
 
-                    break;
-                case 'Custom':
-                    if (_.nonNil(_.get(collision, 'Custom.class')))
-                        colSpi = new EmptyBean(collision.Custom.class);
+        switch (_.get(eventStorage, 'kind')) {
+            case 'Memory':
+                eventStorageBean = new Bean('org.apache.ignite.spi.eventstorage.memory.MemoryEventStorageSpi', 'eventStorage', eventStorage.Memory, clusterDflts.eventStorage.Memory);
 
-                    break;
-                default:
-                    return cfg;
-            }
+                eventStorageBean.intProperty('expireAgeMs')
+                    .intProperty('expireCount')
+                    .emptyBeanProperty('filter');
 
-            if (_.nonNil(colSpi))
-                cfg.beanProperty('collisionSpi', colSpi);
+                break;
 
-            return cfg;
-        }
+            case 'Custom':
+                const className = _.get(eventStorage, 'Custom.className');
 
-        // Generate communication group.
-        static clusterCommunication(cluster, cfg = this.igniteConfigurationBean(cluster)) {
-            const commSpi = new Bean('org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi', 'communicationSpi',
-                cluster.communication, clusterDflts.communication);
+                if (className)
+                    eventStorageBean = new EmptyBean(className);
 
-            commSpi.emptyBeanProperty('listener')
-                .stringProperty('localAddress')
-                .intProperty('localPort')
-                .intProperty('localPortRange')
-                .intProperty('sharedMemoryPort')
-                .intProperty('directBuffer')
-                .intProperty('directSendBuffer')
-                .intProperty('idleConnectionTimeout')
-                .intProperty('connectTimeout')
-                .intProperty('maxConnectTimeout')
-                .intProperty('reconnectCount')
-                .intProperty('socketSendBuffer')
-                .intProperty('socketReceiveBuffer')
-                .intProperty('messageQueueLimit')
-                .intProperty('slowClientQueueLimit')
-                .intProperty('tcpNoDelay')
-                .intProperty('ackSendThreshold')
-                .intProperty('unacknowledgedMessagesBufferSize')
-                .intProperty('socketWriteTimeout')
-                .intProperty('selectorsCount')
-                .emptyBeanProperty('addressResolver');
-
-            if (commSpi.nonEmpty())
-                cfg.beanProperty('communicationSpi', commSpi);
-
-            cfg.intProperty('networkTimeout')
-                .intProperty('networkSendRetryDelay')
-                .intProperty('networkSendRetryCount')
-                .intProperty('discoveryStartupDelay');
+                break;
 
-            return cfg;
+            default:
+                // No-op.
         }
 
-        // Generate REST access configuration.
-        static clusterConnector(connector, cfg = this.igniteConfigurationBean()) {
-            const connCfg = new Bean('org.apache.ignite.configuration.ConnectorConfiguration',
-                'connectorConfiguration', connector, clusterDflts.connector);
-
-            if (connCfg.valueOf('enabled')) {
-                connCfg.pathProperty('jettyPath')
-                    .stringProperty('host')
-                    .intProperty('port')
-                    .intProperty('portRange')
-                    .intProperty('idleTimeout')
-                    .intProperty('idleQueryCursorTimeout')
-                    .intProperty('idleQueryCursorCheckFrequency')
-                    .intProperty('receiveBufferSize')
-                    .intProperty('sendBufferSize')
-                    .intProperty('sendQueueLimit')
-                    .intProperty('directBuffer')
-                    .intProperty('noDelay')
-                    .intProperty('selectorCount')
-                    .intProperty('threadPoolSize')
-                    .emptyBeanProperty('messageInterceptor')
-                    .stringProperty('secretKey');
-
-                if (connCfg.valueOf('sslEnabled')) {
-                    connCfg.intProperty('sslClientAuth')
-                        .emptyBeanProperty('sslFactory');
-                }
-
-                if (connCfg.nonEmpty())
-                    cfg.beanProperty('connectorConfiguration', connCfg);
-            }
+        if (eventStorageBean && eventStorageBean.nonEmpty())
+            cfg.beanProperty('eventStorageSpi', eventStorageBean);
 
-            return cfg;
-        }
+        if (_.nonEmpty(cluster.includeEventTypes))
+            cfg.eventTypes('evts', 'includeEventTypes', cluster.includeEventTypes);
 
-        // Generate deployment group.
-        static clusterDeployment(cluster, cfg = this.igniteConfigurationBean(cluster)) {
-            cfg.enumProperty('deploymentMode')
-                .boolProperty('peerClassLoadingEnabled');
+        return cfg;
+    }
 
-            if (cfg.valueOf('peerClassLoadingEnabled')) {
-                cfg.intProperty('peerClassLoadingMissedResourcesCacheSize')
-                    .intProperty('peerClassLoadingThreadPoolSize')
-                    .varArgProperty('p2pLocClsPathExcl', 'peerClassLoadingLocalClassPathExclude',
-                       cluster.peerClassLoadingLocalClassPathExclude);
-            }
+    // Generate failover group.
+    static clusterFailover(cluster, cfg = this.igniteConfigurationBean(cluster)) {
+        const spis = [];
 
-            return cfg;
-        }
+        _.forEach(cluster.failoverSpi, (spi) => {
+            let failoverSpi;
 
-        // Generate discovery group.
-        static clusterDiscovery(discovery, cfg = this.igniteConfigurationBean(), discoSpi = this.discoveryConfigurationBean(discovery)) {
-            discoSpi.stringProperty('localAddress')
-                .intProperty('localPort')
-                .intProperty('localPortRange')
-                .emptyBeanProperty('addressResolver')
-                .intProperty('socketTimeout')
-                .intProperty('ackTimeout')
-                .intProperty('maxAckTimeout')
-                .intProperty('networkTimeout')
-                .intProperty('joinTimeout')
-                .intProperty('threadPriority')
-                .intProperty('heartbeatFrequency')
-                .intProperty('maxMissedHeartbeats')
-                .intProperty('maxMissedClientHeartbeats')
-                .intProperty('topHistorySize')
-                .emptyBeanProperty('listener')
-                .emptyBeanProperty('dataExchange')
-                .emptyBeanProperty('metricsProvider')
-                .intProperty('reconnectCount')
-                .intProperty('statisticsPrintFrequency')
-                .intProperty('ipFinderCleanFrequency')
-                .emptyBeanProperty('authenticator')
-                .intProperty('forceServerMode')
-                .intProperty('clientReconnectDisabled');
-
-            if (discoSpi.nonEmpty())
-                cfg.beanProperty('discoverySpi', discoSpi);
-
-            return discoSpi;
-        }
+            switch (_.get(spi, 'kind')) {
+                case 'JobStealing':
+                    failoverSpi = new Bean('org.apache.ignite.spi.failover.jobstealing.JobStealingFailoverSpi',
+                        'failoverSpi', spi.JobStealing, clusterDflts.failoverSpi.JobStealing);
 
-        // Generate events group.
-        static clusterEvents(cluster, cfg = this.igniteConfigurationBean(cluster)) {
-            const eventStorage = cluster.eventStorage;
+                    failoverSpi.intProperty('maximumFailoverAttempts');
 
-            let eventStorageBean = null;
+                    break;
+                case 'Never':
+                    failoverSpi = new Bean('org.apache.ignite.spi.failover.never.NeverFailoverSpi',
+                        'failoverSpi', spi.Never);
 
-            switch (_.get(eventStorage, 'kind')) {
-                case 'Memory':
-                    eventStorageBean = new Bean('org.apache.ignite.spi.eventstorage.memory.MemoryEventStorageSpi', 'eventStorage', eventStorage.Memory, clusterDflts.eventStorage.Memory);
+                    break;
+                case 'Always':
+                    failoverSpi = new Bean('org.apache.ignite.spi.failover.always.AlwaysFailoverSpi',
+                        'failoverSpi', spi.Always, clusterDflts.failoverSpi.Always);
 
-                    eventStorageBean.intProperty('expireAgeMs')
-                        .intProperty('expireCount')
-                        .emptyBeanProperty('filter');
+                    failoverSpi.intProperty('maximumFailoverAttempts');
 
                     break;
-
                 case 'Custom':
-                    const className = _.get(eventStorage, 'Custom.className');
+                    const className = _.get(spi, 'Custom.class');
 
                     if (className)
-                        eventStorageBean = new EmptyBean(className);
+                        failoverSpi = new EmptyBean(className);
 
                     break;
-
                 default:
                     // No-op.
             }
 
-            if (eventStorageBean && eventStorageBean.nonEmpty())
-                cfg.beanProperty('eventStorageSpi', eventStorageBean);
-
-            if (_.nonEmpty(cluster.includeEventTypes))
-                cfg.eventTypes('evts', 'includeEventTypes', cluster.includeEventTypes);
-
-            return cfg;
-        }
-
-        // Generate failover group.
-        static clusterFailover(cluster, cfg = this.igniteConfigurationBean(cluster)) {
-            const spis = [];
-
-            _.forEach(cluster.failoverSpi, (spi) => {
-                let failoverSpi;
+            if (failoverSpi)
+                spis.push(failoverSpi);
+        });
 
-                switch (_.get(spi, 'kind')) {
-                    case 'JobStealing':
-                        failoverSpi = new Bean('org.apache.ignite.spi.failover.jobstealing.JobStealingFailoverSpi',
-                            'failoverSpi', spi.JobStealing, clusterDflts.failoverSpi.JobStealing);
+        if (spis.length)
+            cfg.arrayProperty('failoverSpi', 'failoverSpi', spis, 'org.apache.ignite.spi.failover.FailoverSpi');
 
-                        failoverSpi.intProperty('maximumFailoverAttempts');
-
-                        break;
-                    case 'Never':
-                        failoverSpi = new Bean('org.apache.ignite.spi.failover.never.NeverFailoverSpi',
-                            'failoverSpi', spi.Never);
-
-                        break;
-                    case 'Always':
-                        failoverSpi = new Bean('org.apache.ignite.spi.failover.always.AlwaysFailoverSpi',
-                            'failoverSpi', spi.Always, clusterDflts.failoverSpi.Always);
+        return cfg;
+    }
 
-                        failoverSpi.intProperty('maximumFailoverAttempts');
+    // Generate load balancing configuration group.
+    static clusterLoadBalancing(cluster, cfg = this.igniteConfigurationBean(cluster)) {
+        const spis = [];
 
-                        break;
-                    case 'Custom':
-                        const className = _.get(spi, 'Custom.class');
+        _.forEach(cluster.loadBalancingSpi, (spi) => {
+            let loadBalancingSpi;
 
-                        if (className)
-                            failoverSpi = new EmptyBean(className);
+            switch (_.get(spi, 'kind')) {
+                case 'RoundRobin':
+                    loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi', 'loadBalancingSpiRR', spi.RoundRobin, clusterDflts.loadBalancingSpi.RoundRobin);
 
-                        break;
-                    default:
-                        // No-op.
-                }
+                    loadBalancingSpi.boolProperty('perTask');
 
-                if (failoverSpi)
-                    spis.push(failoverSpi);
-            });
+                    break;
+                case 'Adaptive':
+                    loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveLoadBalancingSpi', 'loadBalancingSpiAdaptive', spi.Adaptive);
 
-            if (spis.length)
-                cfg.arrayProperty('failoverSpi', 'failoverSpi', spis, 'org.apache.ignite.spi.failover.FailoverSpi');
+                    let probeBean;
 
-            return cfg;
-        }
+                    switch (_.get(spi, 'Adaptive.loadProbe.kind')) {
+                        case 'Job':
+                            probeBean = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveJobCountLoadProbe', 'jobProbe', spi.Adaptive.loadProbe.Job, clusterDflts.loadBalancingSpi.Adaptive.loadProbe.Job);
 
-        // Generate load balancing configuration group.
-        static clusterLoadBalancing(cluster, cfg = this.igniteConfigurationBean(cluster)) {
-            const spis = [];
+                            probeBean.boolProperty('useAverage');
 
-            _.forEach(cluster.loadBalancingSpi, (spi) => {
-                let loadBalancingSpi;
+                            break;
+                        case 'CPU':
+                            probeBean = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveCpuLoadProbe', 'cpuProbe', spi.Adaptive.loadProbe.CPU, clusterDflts.loadBalancingSpi.Adaptive

<TRUNCATED>

[25/50] [abbrv] ignite git commit: Web console beta-7.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/services/Messages.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/services/Messages.service.js b/modules/web-console/frontend/app/services/Messages.service.js
index e679488..fefdae9 100644
--- a/modules/web-console/frontend/app/services/Messages.service.js
+++ b/modules/web-console/frontend/app/services/Messages.service.js
@@ -24,6 +24,9 @@ export default ['IgniteMessages', ['$alert', ($alert) => {
         prefix = prefix || '';
 
         if (err) {
+            if (err.hasOwnProperty('data'))
+                err = err.data;
+
             if (err.hasOwnProperty('message'))
                 return prefix + err.message;
 
@@ -38,26 +41,26 @@ export default ['IgniteMessages', ['$alert', ($alert) => {
             msgModal.hide();
     };
 
-    const _showMessage = (err, type, duration, icon) => {
+    const _showMessage = (message, err, type, duration) => {
         hideAlert();
 
-        const title = errorMessage(null, err);
+        const title = err ? errorMessage(message, err) : errorMessage(null, message);
 
         msgModal = $alert({type, title, duration});
 
-        msgModal.$scope.icon = icon;
+        msgModal.$scope.icon = `icon-${type}`;
     };
 
     return {
         errorMessage,
         hideAlert,
-        showError(err) {
-            _showMessage(err, 'danger', 10, 'fa-exclamation-triangle');
+        showError(message, err) {
+            _showMessage(message, err, 'danger', 10);
 
             return false;
         },
-        showInfo(err) {
-            _showMessage(err, 'success', 3, 'fa-check-circle-o');
+        showInfo(message) {
+            _showMessage(message, null, 'success', 3);
         }
     };
 }]];

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/controllers/admin-controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/controllers/admin-controller.js b/modules/web-console/frontend/controllers/admin-controller.js
index 7004301..cf7fd71 100644
--- a/modules/web-console/frontend/controllers/admin-controller.js
+++ b/modules/web-console/frontend/controllers/admin-controller.js
@@ -15,79 +15,220 @@
  * limitations under the License.
  */
 
+const ICON_SORT = '<span ui-grid-one-bind-id-grid="col.uid + \'-sortdir-text\'" ui-grid-visible="col.sort.direction" aria-label="Sort Descending"><i ng-class="{ \'ui-grid-icon-up-dir\': col.sort.direction == asc, \'ui-grid-icon-down-dir\': col.sort.direction == desc, \'ui-grid-icon-blank\': !col.sort.direction }" title="" aria-hidden="true"></i></span>';
+
+const CLUSTER_HEADER_TEMPLATE = `<div class='ui-grid-cell-contents' bs-tooltip data-title='{{ col.headerTooltip(col) }}' data-placement='top'><i class='fa fa-sitemap'></i>${ICON_SORT}</div>`;
+const MODEL_HEADER_TEMPLATE = `<div class='ui-grid-cell-contents' bs-tooltip data-title='{{ col.headerTooltip(col) }}' data-placement='top'><i class='fa fa-object-group'></i>${ICON_SORT}</div>`;
+const CACHE_HEADER_TEMPLATE = `<div class='ui-grid-cell-contents' bs-tooltip data-title='{{ col.headerTooltip(col) }}' data-placement='top'><i class='fa fa-database'></i>${ICON_SORT}</div>`;
+const IGFS_HEADER_TEMPLATE = `<div class='ui-grid-cell-contents' bs-tooltip data-title='{{ col.headerTooltip(col) }}' data-placement='top'><i class='fa fa-folder-o'></i>${ICON_SORT}</div>`;
+
+const ACTIONS_TEMPLATE = `
+<div class='text-center ui-grid-cell-actions'>
+    <a class='btn btn-default dropdown-toggle' bs-dropdown='' ng-show='row.entity._id != $root.user._id' data-placement='bottom-right' data-container='.panel'>
+        <i class='fa fa-gear'></i>&nbsp;
+        <span class='caret'></span>
+    </a>
+    <ul class='dropdown-menu' role='menu'>
+        <li>
+            <a ng-click='grid.api.becomeUser(row.entity)'>Become this user</a>
+        </li>
+        <li>
+            <a ng-click='grid.api.toggleAdmin(row.entity)' ng-if='row.entity.admin && row.entity._id !== $root.user._id'>Revoke admin</a>
+            <a ng-click='grid.api.toggleAdmin(row.entity)' ng-if='!row.entity.admin && row.entity._id !== $root.user._id'>Grant admin</a>
+        </li>
+        <li>
+            <a ng-click='grid.api.removeUser(row.entity)'>Remove user</a>
+        </li>
+</div>`;
+
+const EMAIL_TEMPLATE = '<div class="ui-grid-cell-contents"><a ng-href="mailto:{{ COL_FIELD }}">{{ COL_FIELD }}</a></div>';
+
 // Controller for Admin screen.
 export default ['adminController', [
-    '$rootScope', '$scope', '$http', '$q', '$state', 'IgniteMessages', 'IgniteConfirm', 'User', 'IgniteNotebookData', 'IgniteCountries',
-    ($rootScope, $scope, $http, $q, $state, Messages, Confirm, User, Notebook, Countries) => {
+    '$rootScope', '$scope', '$http', '$q', '$state', '$filter', 'uiGridConstants', 'IgniteMessages', 'IgniteConfirm', 'User', 'IgniteNotebookData', 'IgniteCountries',
+    ($rootScope, $scope, $http, $q, $state, $filter, uiGridConstants, Messages, Confirm, User, Notebook, Countries) => {
         $scope.users = null;
 
-        const _reloadUsers = () => {
-            $http.post('/api/v1/admin/list')
-                .success((users) => {
-                    $scope.users = users;
+        const companySelectOptions = [];
+        const countrySelectOptions = [];
 
-                    _.forEach($scope.users, (user) => {
-                        user.userName = user.firstName + ' ' + user.lastName;
-                        user.countryCode = Countries.getByName(user.country).code;
-                        user.label = user.userName + ' ' + user.email + ' ' +
-                            (user.company || '') + ' ' + (user.countryCode || '');
-                    });
-                })
-                .error(Messages.showError);
-        };
+        const COLUMNS_DEFS = [
+            {displayName: 'Actions', cellTemplate: ACTIONS_TEMPLATE, field: 'test', minWidth: 80, width: 80, enableFiltering: false, enableSorting: false},
+            {displayName: 'User', field: 'userName', minWidth: 65, enableFiltering: true, filter: { placeholder: 'Filter by name...' }},
+            {displayName: 'Email', field: 'email', cellTemplate: EMAIL_TEMPLATE, minWidth: 160, enableFiltering: true, filter: { placeholder: 'Filter by email...' }},
+            {displayName: 'Company', field: 'company', minWidth: 160, filter: {
+                selectOptions: companySelectOptions, type: uiGridConstants.filter.SELECT, condition: uiGridConstants.filter.EXACT }
+            },
+            {displayName: 'Country', field: 'countryCode', minWidth: 80, filter: {
+                selectOptions: countrySelectOptions, type: uiGridConstants.filter.SELECT, condition: uiGridConstants.filter.EXACT }
+            },
+            {displayName: 'Last login', field: 'lastLogin', cellFilter: 'date:"medium"', minWidth: 175, width: 175, enableFiltering: false, sort: { direction: 'desc', priority: 0 }},
+            {displayName: 'Clusters count', headerCellTemplate: CLUSTER_HEADER_TEMPLATE, field: '_clusters', type: 'number', headerTooltip: 'Clusters count', minWidth: 50, width: 50, enableFiltering: false},
+            {displayName: 'Models count', headerCellTemplate: MODEL_HEADER_TEMPLATE, field: '_models', type: 'number', headerTooltip: 'Models count', minWidth: 50, width: 50, enableFiltering: false},
+            {displayName: 'Caches count', headerCellTemplate: CACHE_HEADER_TEMPLATE, field: '_caches', type: 'number', headerTooltip: 'Caches count', minWidth: 50, width: 50, enableFiltering: false},
+            {displayName: 'IGFS count', headerCellTemplate: IGFS_HEADER_TEMPLATE, field: '_igfs', type: 'number', headerTooltip: 'IGFS count', minWidth: 50, width: 50, enableFiltering: false}
+        ];
 
-        _reloadUsers();
+        const ctrl = $scope.ctrl = {};
 
-        $scope.becomeUser = function(user) {
+        const becomeUser = function(user) {
             $http.get('/api/v1/admin/become', { params: {viewedUserId: user._id}})
-                .catch(({data}) => Promise.reject(data))
                 .then(() => User.load())
-                .then((becomeUser) => {
-                    $rootScope.$broadcast('user', becomeUser);
-
-                    $state.go('base.configuration.clusters');
-                })
+                .then(() => $state.go('base.configuration.clusters'))
                 .then(() => Notebook.load())
                 .catch(Messages.showError);
         };
 
-        $scope.removeUser = (user) => {
-            Confirm.confirm('Are you sure you want to remove user: "' + user.userName + '"?')
+        const removeUser = (user) => {
+            Confirm.confirm(`Are you sure you want to remove user: "${user.userName}"?`)
                 .then(() => {
                     $http.post('/api/v1/admin/remove', {userId: user._id})
-                        .success(() => {
+                        .then(() => {
                             const i = _.findIndex($scope.users, (u) => u._id === user._id);
 
                             if (i >= 0)
                                 $scope.users.splice(i, 1);
 
-                            Messages.showInfo('User has been removed: "' + user.userName + '"');
+                            Messages.showInfo(`User has been removed: "${user.userName}"`);
                         })
-                        .error((err, status) => {
+                        .catch(({data, status}) => {
                             if (status === 503)
-                                Messages.showInfo(err);
+                                Messages.showInfo(data);
                             else
-                                Messages.showError(Messages.errorMessage('Failed to remove user: ', err));
+                                Messages.showError('Failed to remove user: ', data);
                         });
                 });
         };
 
-        $scope.toggleAdmin = (user) => {
+        const toggleAdmin = (user) => {
             if (user.adminChanging)
                 return;
 
             user.adminChanging = true;
 
             $http.post('/api/v1/admin/save', {userId: user._id, adminFlag: !user.admin})
-                .success(() => {
+                .then(() => {
                     user.admin = !user.admin;
 
-                    Messages.showInfo('Admin right was successfully toggled for user: "' + user.userName + '"');
+                    Messages.showInfo(`Admin right was successfully toggled for user: "${user.userName}"`);
                 })
-                .error((err) => {
-                    Messages.showError(Messages.errorMessage('Failed to toggle admin right for user: ', err));
+                .catch((res) => {
+                    Messages.showError('Failed to toggle admin right for user: ', res);
                 })
                 .finally(() => user.adminChanging = false);
         };
+
+
+        ctrl.gridOptions = {
+            data: [],
+            columnVirtualizationThreshold: 30,
+            columnDefs: COLUMNS_DEFS,
+            categories: [
+                {name: 'Actions', visible: true, selectable: true},
+                {name: 'User', visible: true, selectable: true},
+                {name: 'Email', visible: true, selectable: true},
+                {name: 'Company', visible: true, selectable: true},
+                {name: 'Country', visible: true, selectable: true},
+                {name: 'Last login', visible: true, selectable: true},
+
+                {name: 'Clusters count', visible: true, selectable: true},
+                {name: 'Models count', visible: true, selectable: true},
+                {name: 'Caches count', visible: true, selectable: true},
+                {name: 'IGFS count', visible: true, selectable: true}
+            ],
+            enableFiltering: true,
+            enableRowSelection: false,
+            enableRowHeaderSelection: false,
+            enableColumnMenus: false,
+            multiSelect: false,
+            modifierKeysToMultiSelect: true,
+            noUnselect: true,
+            flatEntityAccess: true,
+            fastWatch: true,
+            onRegisterApi: (api) => {
+                ctrl.gridApi = api;
+
+                api.becomeUser = becomeUser;
+                api.removeUser = removeUser;
+                api.toggleAdmin = toggleAdmin;
+            }
+        };
+
+        /**
+         * Set grid height.
+         *
+         * @param {Number} rows Rows count.
+         * @private
+         */
+        const adjustHeight = (rows) => {
+            const height = Math.min(rows, 20) * 30 + 75;
+
+            // Remove header height.
+            ctrl.gridApi.grid.element.css('height', height + 'px');
+
+            ctrl.gridApi.core.handleWindowResize();
+        };
+
+        const usersToFilterOptions = (column) => {
+            return _.sortBy(
+                _.map(
+                    _.groupBy($scope.users, (usr) => {
+                        const fld = usr[column];
+
+                        return _.isNil(fld) ? fld : fld.toUpperCase();
+                    }),
+                    (arr, value) => ({label: `${_.head(arr)[column] || 'Not set'} (${arr.length})`, value})
+                ),
+                'value');
+        };
+
+        const _reloadUsers = () => {
+            $http.post('/api/v1/admin/list')
+                .then(({ data }) => {
+                    $scope.users = data;
+
+                    companySelectOptions.length = 0;
+                    countrySelectOptions.length = 0;
+
+                    _.forEach($scope.users, (user) => {
+                        user.userName = user.firstName + ' ' + user.lastName;
+                        user.countryCode = Countries.getByName(user.country).code;
+
+                        user._clusters = user.counters.clusters;
+                        user._models = user.counters.models;
+                        user._caches = user.counters.caches;
+                        user._igfs = user.counters.igfs;
+                    });
+
+                    companySelectOptions.push(...usersToFilterOptions('company'));
+                    countrySelectOptions.push(...usersToFilterOptions('countryCode'));
+
+                    $scope.ctrl.gridOptions.data = data;
+
+                    adjustHeight(data.length);
+                })
+                .catch(Messages.showError);
+        };
+
+        _reloadUsers();
+
+        const _enableColumns = (categories, visible) => {
+            _.forEach(categories, (cat) => {
+                cat.visible = visible;
+
+                _.forEach(ctrl.gridOptions.columnDefs, (col) => {
+                    if (col.displayName === cat.name)
+                        col.visible = visible;
+                });
+            });
+
+            ctrl.gridApi.grid.refresh();
+        };
+
+        const _selectableColumns = () => _.filter(ctrl.gridOptions.categories, (cat) => cat.selectable);
+
+        ctrl.toggleColumns = (category, visible) => _enableColumns([category], visible);
+        ctrl.selectAllColumns = () => _enableColumns(_selectableColumns(), true);
+        ctrl.clearAllColumns = () => _enableColumns(_selectableColumns(), false);
     }
 ]];

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/controllers/caches-controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/controllers/caches-controller.js b/modules/web-console/frontend/controllers/caches-controller.js
index 8c01173..e7521b5 100644
--- a/modules/web-console/frontend/controllers/caches-controller.js
+++ b/modules/web-console/frontend/controllers/caches-controller.js
@@ -467,14 +467,14 @@ export default ['cachesController', [
         // Save cache in database.
         function save(item) {
             $http.post('/api/v1/configuration/caches/save', item)
-                .success(function(_id) {
+                .then(({data}) => {
+                    const _id = data;
+
                     item.label = _cacheLbl(item);
 
                     $scope.ui.inputForm.$setPristine();
 
-                    const idx = _.findIndex($scope.caches, function(cache) {
-                        return cache._id === _id;
-                    });
+                    const idx = _.findIndex($scope.caches, {_id});
 
                     if (idx >= 0)
                         _.assign($scope.caches[idx], item);
@@ -487,21 +487,21 @@ export default ['cachesController', [
                         if (_.includes(item.clusters, cluster.value))
                             cluster.caches = _.union(cluster.caches, [_id]);
                         else
-                            _.remove(cluster.caches, (id) => id === _id);
+                            _.pull(cluster.caches, _id);
                     });
 
                     _.forEach($scope.domains, (domain) => {
                         if (_.includes(item.domains, domain.value))
                             domain.meta.caches = _.union(domain.meta.caches, [_id]);
                         else
-                            _.remove(domain.meta.caches, (id) => id === _id);
+                            _.pull(domain.meta.caches, _id);
                     });
 
                     $scope.selectItem(item);
 
                     Messages.showInfo('Cache "' + item.name + '" saved.');
                 })
-                .error(Messages.showError);
+                .catch(Messages.showError);
         }
 
         // Save cache.
@@ -559,7 +559,7 @@ export default ['cachesController', [
                     const _id = selectedItem._id;
 
                     $http.post('/api/v1/configuration/caches/remove', {_id})
-                        .success(function() {
+                        .then(() => {
                             Messages.showInfo('Cache has been removed: ' + selectedItem.name);
 
                             const caches = $scope.caches;
@@ -582,7 +582,7 @@ export default ['cachesController', [
                                 _.forEach($scope.domains, (domain) => _.remove(domain.meta.caches, (id) => id === _id));
                             }
                         })
-                        .error(Messages.showError);
+                        .catch(Messages.showError);
                 });
         };
 
@@ -591,7 +591,7 @@ export default ['cachesController', [
             Confirm.confirm('Are you sure you want to remove all caches?')
                 .then(function() {
                     $http.post('/api/v1/configuration/caches/remove/all')
-                        .success(function() {
+                        .then(() => {
                             Messages.showInfo('All caches have been removed');
 
                             $scope.caches = [];
@@ -603,7 +603,7 @@ export default ['cachesController', [
                             $scope.ui.inputForm.$error = {};
                             $scope.ui.inputForm.$setPristine();
                         })
-                        .error(Messages.showError);
+                        .catch(Messages.showError);
                 });
         };
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/controllers/clusters-controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/controllers/clusters-controller.js b/modules/web-console/frontend/controllers/clusters-controller.js
index f92a2f1..7f90b90 100644
--- a/modules/web-console/frontend/controllers/clusters-controller.js
+++ b/modules/web-console/frontend/controllers/clusters-controller.js
@@ -17,7 +17,7 @@
 
 // Controller for Clusters screen.
 export default ['clustersController', [
-    '$rootScope', '$scope', '$http', '$state', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'igniteEventGroups', 'DemoInfo', 'IgniteLegacyTable', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils',
+    '$rootScope', '$scope', '$http', '$state', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteEventGroups', 'DemoInfo', 'IgniteLegacyTable', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils',
     function($root, $scope, $http, $state, $timeout, LegacyUtils, Messages, Confirm, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, igniteEventGroups, DemoInfo, LegacyTable, Resource, ErrorPopover, FormUtils) {
         UnsavedChangesGuard.install($scope);
 
@@ -31,6 +31,12 @@ export default ['clustersController', [
             cacheKeyConfiguration: [],
             communication: {},
             connector: {},
+            deploymentSpi: {
+                URI: {
+                    uriList: [],
+                    scanners: []
+                }
+            },
             discovery: {
                 Cloud: {
                     regions: [],
@@ -38,6 +44,7 @@ export default ['clustersController', [
                 }
             },
             marshaller: {},
+            peerClassLoadingLocalClassPathExclude: [],
             sslContextFactory: {
                 trustManagers: []
             },
@@ -276,6 +283,16 @@ export default ['clustersController', [
 
                     if (!cluster.eventStorage)
                         cluster.eventStorage = { kind: 'Memory' };
+
+                    if (!cluster.peerClassLoadingLocalClassPathExclude)
+                        cluster.peerClassLoadingLocalClassPathExclude = [];
+
+                    if (!cluster.deploymentSpi) {
+                        cluster.deploymentSpi = {URI: {
+                            uriList: [],
+                            scanners: []
+                        }};
+                    }
                 });
 
                 if ($state.params.linkId)
@@ -699,17 +716,20 @@ export default ['clustersController', [
         // Save cluster in database.
         function save(item) {
             $http.post('/api/v1/configuration/clusters/save', item)
-                .success(function(_id) {
+                .then(({data}) => {
+                    const _id = data;
+
                     item.label = _clusterLbl(item);
 
                     $scope.ui.inputForm.$setPristine();
 
-                    const idx = _.findIndex($scope.clusters, (cluster) => cluster._id === _id);
+                    const idx = _.findIndex($scope.clusters, {_id});
 
                     if (idx >= 0)
                         _.assign($scope.clusters[idx], item);
                     else {
                         item._id = _id;
+
                         $scope.clusters.push(item);
                     }
 
@@ -717,21 +737,21 @@ export default ['clustersController', [
                         if (_.includes(item.caches, cache.value))
                             cache.cache.clusters = _.union(cache.cache.clusters, [_id]);
                         else
-                            _.remove(cache.cache.clusters, (id) => id === _id);
+                            _.pull(cache.cache.clusters, _id);
                     });
 
                     _.forEach($scope.igfss, (igfs) => {
                         if (_.includes(item.igfss, igfs.value))
                             igfs.igfs.clusters = _.union(igfs.igfs.clusters, [_id]);
                         else
-                            _.remove(igfs.igfs.clusters, (id) => id === _id);
+                            _.pull(igfs.igfs.clusters, _id);
                     });
 
                     $scope.selectItem(item);
 
-                    Messages.showInfo('Cluster "' + item.name + '" saved.');
+                    Messages.showInfo(`Cluster "${item.name}" saved.`);
                 })
-                .error(Messages.showError);
+                .catch(Messages.showError);
         }
 
         // Save cluster.
@@ -774,7 +794,7 @@ export default ['clustersController', [
                     const _id = selectedItem._id;
 
                     $http.post('/api/v1/configuration/clusters/remove', {_id})
-                        .success(function() {
+                        .then(() => {
                             Messages.showInfo('Cluster has been removed: ' + selectedItem.name);
 
                             const clusters = $scope.clusters;
@@ -795,7 +815,7 @@ export default ['clustersController', [
                                 _.forEach($scope.igfss, (igfs) => _.remove(igfs.igfs.clusters, (id) => id === _id));
                             }
                         })
-                        .error(Messages.showError);
+                        .catch(Messages.showError);
                 });
         };
 
@@ -804,7 +824,7 @@ export default ['clustersController', [
             Confirm.confirm('Are you sure you want to remove all clusters?')
                 .then(function() {
                     $http.post('/api/v1/configuration/clusters/remove/all')
-                        .success(() => {
+                        .then(() => {
                             Messages.showInfo('All clusters have been removed');
 
                             $scope.clusters = [];
@@ -816,7 +836,7 @@ export default ['clustersController', [
                             $scope.ui.inputForm.$error = {};
                             $scope.ui.inputForm.$setPristine();
                         })
-                        .error(Messages.showError);
+                        .catch(Messages.showError);
                 });
         };
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/controllers/domains-controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/controllers/domains-controller.js b/modules/web-console/frontend/controllers/domains-controller.js
index 2d7b875..303110e 100644
--- a/modules/web-console/frontend/controllers/domains-controller.js
+++ b/modules/web-console/frontend/controllers/domains-controller.js
@@ -756,15 +756,15 @@ export default ['domainsController', [
                 Loading.start('importDomainFromDb');
 
                 $http.post('/api/v1/configuration/domains/save/batch', batch)
-                    .success(function(savedBatch) {
+                    .then(({data}) => {
                         let lastItem;
                         const newItems = [];
 
-                        _.forEach(_mapCaches(savedBatch.generatedCaches), function(cache) {
+                        _.forEach(_mapCaches(data.generatedCaches), function(cache) {
                             $scope.caches.push(cache);
                         });
 
-                        _.forEach(savedBatch.savedDomains, function(savedItem) {
+                        _.forEach(data.savedDomains, function(savedItem) {
                             const idx = _.findIndex($scope.domains, function(domain) {
                                 return domain._id === savedItem._id;
                             });
@@ -792,7 +792,7 @@ export default ['domainsController', [
 
                         $scope.ui.showValid = true;
                     })
-                    .error(Messages.showError)
+                    .catch(Messages.showError)
                     .finally(() => {
                         Loading.finish('importDomainFromDb');
 
@@ -1382,10 +1382,10 @@ export default ['domainsController', [
                 item.kind = 'store';
 
             $http.post('/api/v1/configuration/domains/save', item)
-                .success(function(res) {
+                .then(({data}) => {
                     $scope.ui.inputForm.$setPristine();
 
-                    const savedMeta = res.savedDomains[0];
+                    const savedMeta = data.savedDomains[0];
 
                     const idx = _.findIndex($scope.domains, function(domain) {
                         return domain._id === savedMeta._id;
@@ -1400,16 +1400,16 @@ export default ['domainsController', [
                         if (_.includes(item.caches, cache.value))
                             cache.cache.domains = _.union(cache.cache.domains, [savedMeta._id]);
                         else
-                            _.remove(cache.cache.domains, (id) => id === savedMeta._id);
+                            _.pull(cache.cache.domains, savedMeta._id);
                     });
 
                     $scope.selectItem(savedMeta);
 
-                    Messages.showInfo('Domain model "' + item.valueType + '" saved.');
+                    Messages.showInfo(`Domain model "${item.valueType}" saved.`);
 
                     _checkShowValidPresentation();
                 })
-                .error(Messages.showError);
+                .catch(Messages.showError);
         }
 
         // Save domain model.
@@ -1469,14 +1469,12 @@ export default ['domainsController', [
                     const _id = selectedItem._id;
 
                     $http.post('/api/v1/configuration/domains/remove', {_id})
-                        .success(function() {
+                        .then(() => {
                             Messages.showInfo('Domain model has been removed: ' + selectedItem.valueType);
 
                             const domains = $scope.domains;
 
-                            const idx = _.findIndex(domains, function(domain) {
-                                return domain._id === _id;
-                            });
+                            const idx = _.findIndex(domains, {_id});
 
                             if (idx >= 0) {
                                 domains.splice(idx, 1);
@@ -1488,12 +1486,12 @@ export default ['domainsController', [
                                 else
                                     $scope.backupItem = emptyDomain;
 
-                                _.forEach($scope.caches, (cache) => _.remove(cache.cache.domains, (id) => id === _id));
+                                _.forEach($scope.caches, (cache) => _.pull(cache.cache.domains, _id));
                             }
 
                             _checkShowValidPresentation();
                         })
-                        .error(Messages.showError);
+                        .catch(Messages.showError);
                 });
         };
 
@@ -1504,7 +1502,7 @@ export default ['domainsController', [
             Confirm.confirm('Are you sure you want to remove all domain models?')
                 .then(function() {
                     $http.post('/api/v1/configuration/domains/remove/all')
-                        .success(function() {
+                        .then(() => {
                             Messages.showInfo('All domain models have been removed');
 
                             $scope.domains = [];
@@ -1516,7 +1514,7 @@ export default ['domainsController', [
                             $scope.ui.inputForm.$error = {};
                             $scope.ui.inputForm.$setPristine();
                         })
-                        .error(Messages.showError);
+                        .catch(Messages.showError);
                 });
         };
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/controllers/igfs-controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/controllers/igfs-controller.js b/modules/web-console/frontend/controllers/igfs-controller.js
index e505f1c..b3c6043 100644
--- a/modules/web-console/frontend/controllers/igfs-controller.js
+++ b/modules/web-console/frontend/controllers/igfs-controller.js
@@ -296,12 +296,12 @@ export default ['igfsController', [
         // Save IGFS in database.
         function save(item) {
             $http.post('/api/v1/configuration/igfs/save', item)
-                .success(function(_id) {
+                .then(({data}) => {
+                    const _id = data;
+
                     $scope.ui.inputForm.$setPristine();
 
-                    const idx = _.findIndex($scope.igfss, function(igfs) {
-                        return igfs._id === _id;
-                    });
+                    const idx = _.findIndex($scope.igfss, {_id});
 
                     if (idx >= 0)
                         _.assign($scope.igfss[idx], item);
@@ -312,9 +312,9 @@ export default ['igfsController', [
 
                     $scope.selectItem(item);
 
-                    Messages.showInfo('IGFS "' + item.name + '" saved.');
+                    Messages.showInfo(`IGFS "${item.name}" saved.`);
                 })
-                .error(Messages.showError);
+                .catch(Messages.showError);
         }
 
         // Save IGFS.
@@ -359,7 +359,7 @@ export default ['igfsController', [
                     const _id = selectedItem._id;
 
                     $http.post('/api/v1/configuration/igfs/remove', {_id})
-                        .success(function() {
+                        .then(() => {
                             Messages.showInfo('IGFS has been removed: ' + selectedItem.name);
 
                             const igfss = $scope.igfss;
@@ -379,7 +379,7 @@ export default ['igfsController', [
                                     $scope.backupItem = emptyIgfs;
                             }
                         })
-                        .error(Messages.showError);
+                        .catch(Messages.showError);
                 });
         };
 
@@ -390,7 +390,7 @@ export default ['igfsController', [
             Confirm.confirm('Are you sure you want to remove all IGFS?')
                 .then(function() {
                     $http.post('/api/v1/configuration/igfs/remove/all')
-                        .success(function() {
+                        .then(() => {
                             Messages.showInfo('All IGFS have been removed');
 
                             $scope.igfss = [];
@@ -398,7 +398,7 @@ export default ['igfsController', [
                             $scope.ui.inputForm.$error = {};
                             $scope.ui.inputForm.$setPristine();
                         })
-                        .error(Messages.showError);
+                        .catch(Messages.showError);
                 });
         };
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/controllers/profile-controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/controllers/profile-controller.js b/modules/web-console/frontend/controllers/profile-controller.js
index fd595d9..87a8805 100644
--- a/modules/web-console/frontend/controllers/profile-controller.js
+++ b/modules/web-console/frontend/controllers/profile-controller.js
@@ -74,7 +74,6 @@ export default ['profileController', [
 
         $scope.saveUser = () => {
             $http.post('/api/v1/profile/save', $scope.user)
-                .catch(({data}) => Promise.reject(data))
                 .then(User.load)
                 .then(() => {
                     if ($scope.expandedPassword)
@@ -89,7 +88,7 @@ export default ['profileController', [
 
                     $root.$broadcast('user', $scope.user);
                 })
-                .catch((err) => Messages.showError(Messages.errorMessage('Failed to save profile: ', err)));
+                .catch((res) => Messages.showError('Failed to save profile: ', res));
         };
     }
 ]];

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js
index 2463d24..7360ac4 100644
--- a/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js
+++ b/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js
@@ -20,7 +20,7 @@ import fs from 'fs';
 import webpack from 'webpack';
 import autoprefixer from 'autoprefixer-core';
 import jade from 'jade';
-import progressPlugin from './plugins/progress';
+import ProgressBarPlugin from 'progress-bar-webpack-plugin';
 import eslintFormatter from 'eslint-friendly-formatter';
 
 import ExtractTextPlugin from 'extract-text-webpack-plugin';
@@ -61,7 +61,6 @@ export default () => {
         // Output system.
         output: {
             path: destDir,
-            publicPath: './',
             filename: '[name].js'
         },
 
@@ -111,8 +110,10 @@ export default () => {
                     loader: 'babel-loader',
                     query: {
                         cacheDirectory: true,
-                        plugins: ['transform-runtime',
-                            'add-module-exports'],
+                        plugins: [
+                            'transform-runtime',
+                            'add-module-exports'
+                        ],
                         presets: ['angular']
 
                     }
@@ -126,10 +127,8 @@ export default () => {
                     loader: development ? `style-loader!${stylesLoader}` : ExtractTextPlugin.extract('style-loader', stylesLoader)
                 },
                 {
-                    test: /\.(woff2|woff|ttf|eot|svg)?(\?v=[0-9]\.[0-9]\.[0-9])?$/,
-                    loaders: [
-                        `${assetsLoader}?name=assets/fonts/[name].[ext]`
-                    ]
+                    test: /\.(ttf|eot|svg|woff(2)?)(\?v=[\d.]+)?(\?[a-z0-9#-]+)?$/,
+                    loaders: [`${assetsLoader}?name=assets/fonts/[name].[ext]`]
                 },
                 {
                     test: /\.(jpe?g|png|gif)$/i,
@@ -186,7 +185,7 @@ export default () => {
                 },
                 favicon
             }),
-            progressPlugin
+            new ProgressBarPlugin()
         ]
     };
 };

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js
index cad9133..34e1f6a 100644
--- a/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js
+++ b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js
@@ -20,9 +20,8 @@ import webpack from 'webpack';
 
 import {destDir, rootDir, srcDir} from '../../paths';
 
-const devServerHost = 'localhost';
+const backendPort = 3000;
 const devServerPort = 9000;
-const devServerUrl = `http://${devServerHost}:${devServerPort}/`;
 
 export default () => {
     const plugins = [
@@ -31,11 +30,10 @@ export default () => {
 
     return {
         entry: {
-            webpack: `webpack-dev-server/client?${devServerUrl}`,
             app: [path.join(srcDir, 'app.js'), 'webpack/hot/only-dev-server']
         },
         output: {
-            publicPath: devServerUrl
+            publicPath: `http://localhost:${devServerPort}/`
         },
         context: rootDir,
         debug: true,
@@ -44,24 +42,22 @@ export default () => {
         devServer: {
             compress: true,
             historyApiFallback: true,
-            publicPath: '/',
             contentBase: destDir,
-            info: true,
             hot: true,
             inline: true,
             proxy: {
                 '/socket.io': {
-                    target: 'http://localhost:3000',
+                    target: `http://localhost:${backendPort}`,
                     changeOrigin: true,
                     ws: true
                 },
                 '/agents': {
-                    target: 'http://localhost:3000',
+                    target: `http://localhost:${backendPort}`,
                     changeOrigin: true,
                     ws: true
                 },
                 '/api/v1/*': {
-                    target: 'http://localhost:3000',
+                    target: `http://localhost:${backendPort}`,
                     changeOrigin: true,
                     pathRewrite: {
                         '^/api/v1': ''

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/production.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/production.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/production.js
index db66720..1194568 100644
--- a/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/production.js
+++ b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/production.js
@@ -37,8 +37,7 @@ export default () => {
         devtool: 'cheap-source-map',
         output: {
             publicPath: '/',
-            filename: '[name].[chunkhash].js',
-            path: destDir
+            filename: '[name].[chunkhash].js'
         },
         plugins
     };

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/gulpfile.babel.js/webpack/plugins/progress.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/gulpfile.babel.js/webpack/plugins/progress.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/plugins/progress.js
deleted file mode 100644
index 5f753c7..0000000
--- a/modules/web-console/frontend/gulpfile.babel.js/webpack/plugins/progress.js
+++ /dev/null
@@ -1,82 +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.
- */
-
-import ProgressPlugin from 'webpack/lib/ProgressPlugin';
-
-let chars = 0;
-let lastState = 0;
-let lastStateTime = 0;
-
-const outputStream = process.stdout;
-
-const _goToLineStart = (nextMessage) => {
-    let str = '';
-
-    for (; chars > nextMessage.length; chars--)
-        str += '\b \b';
-
-    chars = nextMessage.length;
-
-    for (let i = 0; i < chars; i++)
-        str += '\b';
-
-    if (str)
-        outputStream.write(str);
-};
-
-export default new ProgressPlugin((percentage, msg) => {
-    let state = msg;
-
-    if (percentage < 1) {
-        percentage = Math.floor(percentage * 100);
-
-        msg = percentage + '% ' + msg;
-
-        if (percentage < 100)
-            msg = ' ' + msg;
-
-        if (percentage < 10)
-            msg = ' ' + msg;
-    }
-
-    state = state.replace(/^\d+\/\d+\s+/, '');
-
-    if (percentage === 0) {
-        lastState = null;
-        lastStateTime = (new Date()).getTime();
-    }
-    else if (state !== lastState || percentage === 1) {
-        const now = (new Date()).getTime();
-
-        if (lastState) {
-            const stateMsg = (now - lastStateTime) + 'ms ' + lastState;
-
-            _goToLineStart(stateMsg);
-
-            outputStream.write(stateMsg + '\n');
-
-            chars = 0;
-        }
-
-        lastState = state;
-        lastStateTime = now;
-    }
-
-    _goToLineStart(msg);
-
-    outputStream.write(msg);
-});

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/package.json
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/package.json b/modules/web-console/frontend/package.json
index b511ca1..fd50d5b 100644
--- a/modules/web-console/frontend/package.json
+++ b/modules/web-console/frontend/package.json
@@ -29,97 +29,99 @@
     "win32"
   ],
   "dependencies": {
-    "angular": "^1.5.5",
-    "angular-acl": "^0.1.7",
-    "angular-animate": "^1.5.5",
-    "angular-aria": "^1.5.5",
-    "angular-cookies": "^1.5.5",
-    "angular-drag-and-drop-lists": "^1.4.0",
-    "angular-gridster": "^0.13.3",
-    "angular-motion": "^0.4.4",
-    "angular-nvd3": "^1.0.7",
-    "angular-retina": "^0.3.13",
-    "angular-sanitize": "^1.5.5",
-    "angular-smart-table": "^2.1.8",
-    "angular-socket-io": "^0.7.0",
-    "angular-strap": "^2.3.8",
-    "angular-touch": "^1.5.5",
-    "angular-tree-control": "^0.2.26",
-    "angular-ui-grid": "^3.1.1",
-    "angular-ui-router": "^0.3.1",
-    "bootstrap-sass": "^3.3.6",
-    "brace": "^0.8.0",
-    "es6-promise": "^3.0.2",
-    "file-saver": "^1.3.2",
-    "font-awesome": "^4.6.3",
-    "glob": "^7.0.3",
-    "jquery": "^3.0.0",
-    "jszip": "^3.0.0",
-    "lodash": "^4.8.2",
-    "nvd3": "^1.8.3",
-    "raleway-webfont": "^3.0.1",
-    "roboto-font": "^0.1.0",
-    "socket.io-client": "^1.4.6",
-    "ui-router-metatags": "^1.0.3"
+    "angular": "~1.5.9",
+    "angular-acl": "~0.1.7",
+    "angular-animate": "~1.5.9",
+    "angular-aria": "~1.5.9",
+    "angular-cookies": "~1.5.9",
+    "angular-drag-and-drop-lists": "~1.4.0",
+    "angular-gridster": "~0.13.3",
+    "angular-motion": "~0.4.4",
+    "angular-nvd3": "~1.0.9",
+    "angular-retina": "~0.3.13",
+    "angular-sanitize": "~1.5.9",
+    "angular-smart-table": "~2.1.8",
+    "angular-socket-io": "~0.7.0",
+    "angular-strap": "~2.3.8",
+    "angular-touch": "~1.5.9",
+    "angular-tree-control": "~0.2.26",
+    "angular-ui-grid": "~3.2.9",
+    "angular-ui-router": "~0.3.1",
+    "bootstrap-sass": "~3.3.6",
+    "brace": "~0.8.0",
+    "es6-promise": "~3.3.1",
+    "file-saver": "~1.3.2",
+    "font-awesome": "~4.7.0",
+    "glob": "~7.1.1",
+    "jquery": "~3.1.1",
+    "jszip": "~3.1.3",
+    "lodash": "~4.17.2",
+    "nvd3": "1.8.4",
+    "raleway-webfont": "~3.0.1",
+    "roboto-font": "~0.1.0",
+    "socket.io-client": "~1.7.2",
+    "ui-router-metatags": "~1.0.3"
   },
   "devDependencies": {
-    "assets-webpack-plugin": "^3.2.0",
-    "autoprefixer-core": "^6.0.1",
-    "babel-core": "^6.7.6",
-    "babel-eslint": "^7.0.0",
-    "babel-loader": "^6.2.4",
-    "babel-plugin-add-module-exports": "^0.2.1",
-    "babel-plugin-transform-builtin-extend": "^1.1.0",
-    "babel-plugin-transform-runtime": "^6.7.5",
-    "babel-polyfill": "^6.7.4",
-    "babel-preset-angular": "^6.0.15",
-    "babel-preset-es2015": "^6.9.0",
-    "babel-runtime": "^6.6.1",
-    "chai": "^3.5.0",
-    "cross-env": "^1.0.7",
-    "css-loader": "^0.23.0",
-    "eslint": "^3.0.0",
-    "eslint-friendly-formatter": "^2.0.5",
-    "eslint-loader": "^1.0.0",
-    "expose-loader": "^0.7.1",
-    "extract-text-webpack-plugin": "^1.0.1",
-    "file-loader": "^0.9.0",
-    "gulp": "^3.9.1",
-    "gulp-eslint": "^3.0.0",
-    "gulp-inject": "^4.0.0",
-    "gulp-jade": "^1.1.0",
-    "gulp-ll": "^1.0.4",
-    "gulp-rimraf": "^0.2.0",
-    "gulp-sequence": "^0.4.1",
-    "gulp-util": "^3.0.7",
-    "html-loader": "^0.4.3",
-    "html-webpack-plugin": "^2.21.0",
-    "jade": "^1.11.0",
+    "assets-webpack-plugin": "~3.5.0",
+    "autoprefixer-core": "~6.0.1",
+    "babel-core": "~6.20.0",
+    "babel-eslint": "~7.0.0",
+    "babel-loader": "~6.2.4",
+    "babel-plugin-add-module-exports": "~0.2.1",
+    "babel-plugin-transform-builtin-extend": "~1.1.0",
+    "babel-plugin-transform-runtime": "~6.15.0",
+    "babel-polyfill": "~6.20.0",
+    "babel-preset-angular": "~6.0.15",
+    "babel-preset-es2015": "~6.18.0",
+    "babel-runtime": "~6.20.0",
+    "chai": "~3.5.0",
+    "cross-env": "~1.0.7",
+    "css-loader": "~0.23.0",
+    "eslint": "~3.12.2",
+    "eslint-friendly-formatter": "~2.0.5",
+    "eslint-loader": "~1.6.1",
+    "expose-loader": "~0.7.1",
+    "extract-text-webpack-plugin": "~1.0.1",
+    "file-loader": "~0.9.0",
+    "gulp": "~3.9.1",
+    "gulp-eslint": "~3.0.0",
+    "gulp-inject": "~4.1.0",
+    "gulp-jade": "~1.1.0",
+    "gulp-ll": "~1.0.4",
+    "gulp-rimraf": "~0.2.0",
+    "gulp-sequence": "~0.4.1",
+    "gulp-util": "~3.0.7",
+    "html-loader": "~0.4.3",
+    "html-webpack-plugin": "~2.24.1",
+    "jade": "~1.11.0",
     "jade-html-loader": "git://github.com/courcelan/jade-html-loader",
-    "jasmine-core": "^2.4.1",
-    "json-loader": "^0.5.4",
-    "karma": "^0.13.22",
-    "karma-babel-preprocessor": "^6.0.1",
-    "karma-jasmine": "^1.0.2",
-    "karma-mocha": "^1.0.1",
-    "karma-mocha-reporter": "^2.2.0",
-    "karma-phantomjs-launcher": "^1.0.0",
-    "karma-teamcity-reporter": "^1.0.0",
-    "karma-webpack": "^1.7.0",
+    "jasmine-core": "~2.5.2",
+    "json-loader": "~0.5.4",
+    "karma": "~0.13.22",
+    "karma-babel-preprocessor": "~6.0.1",
+    "karma-jasmine": "~1.1.0",
+    "karma-mocha": "~1.3.0",
+    "karma-mocha-reporter": "~2.2.0",
+    "karma-phantomjs-launcher": "~1.0.0",
+    "karma-teamcity-reporter": "~1.0.0",
+    "karma-webpack": "~1.8.0",
     "mocha": "~2.5.3",
-    "mocha-teamcity-reporter": "^1.0.0",
-    "morgan": "^1.7.0",
-    "ngtemplate-loader": "^1.3.1",
-    "node-sass": "^3.4.2",
-    "phantomjs-prebuilt": "^2.1.7",
-    "postcss-loader": "^0.9.1",
-    "require-dir": "^0.3.0",
-    "resolve-url-loader": "^1.4.3",
-    "sass-loader": "^3.1.1",
-    "style-loader": "^0.13.1",
-    "url": "^0.11.0",
-    "url-loader": "^0.5.6",
-    "webpack": "^1.13.1",
-    "webpack-dev-server": "^1.15.0"
+    "mocha-teamcity-reporter": "~1.1.1",
+    "morgan": "~1.7.0",
+    "ngtemplate-loader": "~1.3.1",
+    "node-sass": "~3.13.1",
+    "phantomjs-prebuilt": "~2.1.7",
+    "postcss-loader": "~0.9.1",
+    "progress-bar-webpack-plugin": "~1.9.0",
+    "require-dir": "~0.3.0",
+    "resolve-url-loader": "~1.6.1",
+    "sass-loader": "~3.1.1",
+    "style-loader": "~0.13.1",
+    "url": "~0.11.0",
+    "url-loader": "~0.5.6",
+    "webpack": "~1.14.0",
+    "webpack-dev-server": "~1.16.2",
+    "worker-loader": "~0.7.1"
   }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/public/images/cache.png
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/images/cache.png b/modules/web-console/frontend/public/images/cache.png
index 83fd987..3ff3103 100644
Binary files a/modules/web-console/frontend/public/images/cache.png and b/modules/web-console/frontend/public/images/cache.png differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/public/images/domains.png
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/images/domains.png b/modules/web-console/frontend/public/images/domains.png
index 39abfcb..41c0470 100644
Binary files a/modules/web-console/frontend/public/images/domains.png and b/modules/web-console/frontend/public/images/domains.png differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/public/images/igfs.png
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/images/igfs.png b/modules/web-console/frontend/public/images/igfs.png
index 47c659e..b62c27b 100644
Binary files a/modules/web-console/frontend/public/images/igfs.png and b/modules/web-console/frontend/public/images/igfs.png differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/public/images/query-chart.png
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/images/query-chart.png b/modules/web-console/frontend/public/images/query-chart.png
index c6e4cce..1b7ef41 100644
Binary files a/modules/web-console/frontend/public/images/query-chart.png and b/modules/web-console/frontend/public/images/query-chart.png differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/public/images/query-metadata.png
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/images/query-metadata.png b/modules/web-console/frontend/public/images/query-metadata.png
index 698cd6e..1b6c73c 100644
Binary files a/modules/web-console/frontend/public/images/query-metadata.png and b/modules/web-console/frontend/public/images/query-metadata.png differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/public/images/query-table.png
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/images/query-table.png b/modules/web-console/frontend/public/images/query-table.png
index 53becda..4d63a68 100644
Binary files a/modules/web-console/frontend/public/images/query-table.png and b/modules/web-console/frontend/public/images/query-table.png differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/public/images/summary.png
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/images/summary.png b/modules/web-console/frontend/public/images/summary.png
index ff88438..fda0abf 100644
Binary files a/modules/web-console/frontend/public/images/summary.png and b/modules/web-console/frontend/public/images/summary.png differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/public/stylesheets/_font-awesome-custom.scss
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/stylesheets/_font-awesome-custom.scss b/modules/web-console/frontend/public/stylesheets/_font-awesome-custom.scss
index 15ee60c..bfa6c6c 100644
--- a/modules/web-console/frontend/public/stylesheets/_font-awesome-custom.scss
+++ b/modules/web-console/frontend/public/stylesheets/_font-awesome-custom.scss
@@ -47,4 +47,25 @@ $fa-font-path: '~font-awesome/fonts';
   @extend .fa-question-circle-o;
 
   cursor: default;
-}
\ No newline at end of file
+}
+
+.icon-note {
+  @extend .fa;
+  @extend .fa-info-circle;
+
+  cursor: default;
+}
+
+.icon-danger {
+  @extend .fa;
+  @extend .fa-exclamation-triangle;
+
+  cursor: default;
+}
+
+.icon-success {
+  @extend .fa;
+  @extend .fa-check-circle-o;
+
+  cursor: default;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/public/stylesheets/form-field.scss
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/stylesheets/form-field.scss b/modules/web-console/frontend/public/stylesheets/form-field.scss
index f126786..ae33d75 100644
--- a/modules/web-console/frontend/public/stylesheets/form-field.scss
+++ b/modules/web-console/frontend/public/stylesheets/form-field.scss
@@ -106,3 +106,40 @@
         @include make-lg-column(8);
     }
 }
+
+.ignite-form-field {
+    &__btn {
+        overflow: hidden;
+
+        border-top-left-radius: 0;
+        border-bottom-left-radius: 0;
+
+        &.btn {
+            float: right;
+            margin-right: 0;
+
+            line-height: 20px;
+        }
+
+        input {
+            position: absolute;
+            left: 100px;
+        }
+
+        input:checked + span {
+            color: $brand-info;
+        }
+    }
+
+    &__btn ~ &__btn {
+        border-right: 0;
+        border-top-right-radius: 0;
+        border-bottom-right-radius: 0;  
+    }
+
+    &__btn ~ .input-tip input {
+        border-right: 0;
+        border-top-right-radius: 0;
+        border-bottom-right-radius: 0;  
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/public/stylesheets/style.scss
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/stylesheets/style.scss b/modules/web-console/frontend/public/stylesheets/style.scss
index 172abf4..4318fc2 100644
--- a/modules/web-console/frontend/public/stylesheets/style.scss
+++ b/modules/web-console/frontend/public/stylesheets/style.scss
@@ -227,7 +227,7 @@ ul.navbar-nav, .sidebar-nav {
         overflow: hidden;
         white-space: nowrap;
         text-overflow: ellipsis;
-        
+
         &:hover,
         &:focus {
             text-decoration: none;
@@ -601,6 +601,10 @@ button.form-control {
 .theme-line .notebook-header {
     border-color: $gray-lighter;
 
+    button:last-child {
+        margin-right: 0;
+    }
+
     h1 {
         padding: 0;
         margin: 0;
@@ -611,7 +615,7 @@ button.form-control {
             overflow: hidden;
             text-overflow: ellipsis;
             white-space: nowrap;
-            margin-top: 5px;
+            height: 24px;
         }
 
         .btn-group {
@@ -637,7 +641,7 @@ button.form-control {
 }
 
 .theme-line .paragraphs {
-    .panel-group .panel + .panel {
+    .panel-group .panel-paragraph + .panel-paragraph {
         margin-top: 30px;
     }
 
@@ -679,8 +683,14 @@ button.form-control {
         line-height: 55px;
     }
 
-    .sql-controls {
+    .panel-collapse {
         border-top: 1px solid $ignite-border-color;
+    }
+
+    .sql-controls {
+        position: relative;
+        top: -1px;
+        border-top: 1px solid #ddd;
 
         padding: 10px 10px;
 
@@ -690,9 +700,50 @@ button.form-control {
         }
 
         label {
-            line-height: 20px !important;
+            line-height: 28px;
             vertical-align: middle;
         }
+
+        .btn {
+            line-height: 20px;
+        }
+
+        .ignite-form-field {
+            margin-right: 10px;
+
+            .ignite-form-field__label {
+                float: left;
+                width: auto;
+                margin-right: 5px;
+                line-height: 28px;
+            }
+
+            .ignite-form-field__label + div {
+                display: block;
+                float: none;
+                width: auto;
+            }
+        }
+
+        .tipLabel .btn {
+            float: right;
+        }
+
+        .pull-right {
+            margin-left: 10px;
+
+            .ignite-form-field {
+                margin-right: -24px;
+
+                label {
+                    margin-left: 5px;
+                }
+            }
+        }
+
+        .col-sm-3 + .tipLabel {
+            margin-left: 0;
+        }
     }
 
     .sql-result {
@@ -1243,6 +1294,17 @@ button.form-control {
     .clickable { cursor: pointer; }
 }
 
+
+.theme-line .summary {
+    .actions-note {
+        i {
+            margin-right: 5px;
+        }
+
+        margin: 15px 0;
+    }
+}
+
 .theme-line .popover.summary-project-structure {
     @extend .popover.settings;
 
@@ -1693,6 +1755,7 @@ th[st-sort] {
 }
 
 .chart-settings-link {
+    margin-top: -2px;
     padding-left: 10px;
     line-height: $input-height;
 
@@ -1991,6 +2054,10 @@ treecontrol.tree-classic {
         margin-right: 0
     }
 
+    .ui-grid-cell-actions {
+        line-height: 28px;
+    }
+
     .no-rows {
         .center-container {
             background: white;
@@ -2226,3 +2293,37 @@ html,body,.splash-screen {
         animation: none 0s;
     }
 }
+
+.admin-page {
+    .panel-heading {
+        border-bottom: 0;
+        padding-bottom: 0;
+
+        cursor: default;
+
+        i {
+            margin-right: 10px;
+        }
+
+        label {
+            cursor: default;
+            line-height: 24px;
+        }
+
+        sub {
+            bottom: 0;
+        }
+    }
+
+    .ui-grid-header-cell input {
+      font-weight: normal;
+    }
+
+    .ui-grid-header-cell input {
+      font-weight: normal;
+    }
+
+    .ui-grid-filter-select {
+        width: calc(100% - 10px);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/test/unit/JavaTypes.test.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/test/unit/JavaTypes.test.js b/modules/web-console/frontend/test/unit/JavaTypes.test.js
index 2df8c6a..49e78cc 100644
--- a/modules/web-console/frontend/test/unit/JavaTypes.test.js
+++ b/modules/web-console/frontend/test/unit/JavaTypes.test.js
@@ -17,11 +17,11 @@
 
 import JavaTypes from '../../app/services/JavaTypes.service.js';
 
-import ClusterDflts from '../../app/modules/configuration/generator/defaults/cluster.provider';
-import CacheDflts from '../../app/modules/configuration/generator/defaults/cache.provider';
-import IgfsDflts from '../../app/modules/configuration/generator/defaults/igfs.provider';
+import ClusterDflts from '../../app/modules/configuration/generator/defaults/Cluster.service';
+import CacheDflts from '../../app/modules/configuration/generator/defaults/Cache.service';
+import IgfsDflts from '../../app/modules/configuration/generator/defaults/IGFS.service';
 
-const INSTANCE = new JavaTypes((new ClusterDflts()).$get[0](), (new CacheDflts()).$get[0](), (new IgfsDflts()).$get[0]());
+const INSTANCE = new JavaTypes(new ClusterDflts(), new CacheDflts(), new IgfsDflts());
 
 import { assert } from 'chai';
 
@@ -58,9 +58,14 @@ suite('JavaTypesTestsSuite', () => {
 
     test('shortClassName', () => {
         assert.equal(INSTANCE.shortClassName('java.math.BigDecimal'), 'BigDecimal');
+        assert.equal(INSTANCE.shortClassName('BigDecimal'), 'BigDecimal');
         assert.equal(INSTANCE.shortClassName('int'), 'int');
         assert.equal(INSTANCE.shortClassName('java.lang.Integer'), 'Integer');
+        assert.equal(INSTANCE.shortClassName('Integer'), 'Integer');
         assert.equal(INSTANCE.shortClassName('java.util.UUID'), 'UUID');
+        assert.equal(INSTANCE.shortClassName('java.sql.Date'), 'Date');
+        assert.equal(INSTANCE.shortClassName('Date'), 'Date');
+        assert.equal(INSTANCE.shortClassName('com.my.Abstract'), 'Abstract');
         assert.equal(INSTANCE.shortClassName('Abstract'), 'Abstract');
     });
 
@@ -113,8 +118,8 @@ suite('JavaTypesTestsSuite', () => {
         assert.equal(INSTANCE.isKeyword(' '), false);
     });
 
-    test('isJavaPrimitive', () => {
-        assert.equal(INSTANCE.isJavaPrimitive('boolean'), true);
+    test('isPrimitive', () => {
+        assert.equal(INSTANCE.isPrimitive('boolean'), true);
     });
 
     test('validUUID', () => {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/test/unit/Version.test.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/test/unit/Version.test.js b/modules/web-console/frontend/test/unit/Version.test.js
index a67fde8..2d75ab5 100644
--- a/modules/web-console/frontend/test/unit/Version.test.js
+++ b/modules/web-console/frontend/test/unit/Version.test.js
@@ -39,7 +39,13 @@ suite('VersionServiceTestsSuite', () => {
     });
 
     test('Version a = b', () => {
-        assert.equal(INSTANCE.compare('1.7.0', '1.7.0'), 0);
+        assert.equal(INSTANCE.compare('1.0.0', '1.0.0'), 0);
+        assert.equal(INSTANCE.compare('1.2.0', '1.2.0'), 0);
+        assert.equal(INSTANCE.compare('1.2.3', '1.2.3'), 0);
+
+        assert.equal(INSTANCE.compare('1.0.0-1', '1.0.0-1'), 0);
+        assert.equal(INSTANCE.compare('1.2.0-1', '1.2.0-1'), 0);
+        assert.equal(INSTANCE.compare('1.2.3-1', '1.2.3-1'), 0);
     });
 
     test('Version a < b', () => {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/views/configuration/domains-import.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/configuration/domains-import.jade b/modules/web-console/frontend/views/configuration/domains-import.jade
index e4f95bc..bbcb391 100644
--- a/modules/web-console/frontend/views/configuration/domains-import.jade
+++ b/modules/web-console/frontend/views/configuration/domains-import.jade
@@ -62,7 +62,10 @@ mixin td-ellipses-lbl(w, lbl)
                             +ignite-form-field-dropdown('Driver JAR:', 'ui.selectedJdbcDriverJar', '"jdbcDriverJar"', false, true, false,
                                 'Choose JDBC driver', '', 'jdbcDriverJars',
                                 'Select appropriate JAR with JDBC driver<br> To add another driver you need to place it into "/jdbc-drivers" folder of Ignite Web Agent<br> Refer to Ignite Web Agent README.txt for for more information'
-                            )(data-container='.modal-domain-import')
+                            )(
+                                data-container='.modal-domain-import'
+                                data-ignite-form-field-input-autofocus='true'
+                            )
                         .settings-row.settings-row_small-label
                             +java-class('JDBC driver:', 'selectedPreset.jdbcDriverClass', '"jdbcDriverClass"', true, true, 'Fully qualified class name of JDBC driver that will be used to connect to database')
                         .settings-row.settings-row_small-label

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/views/configuration/summary.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/configuration/summary.jade b/modules/web-console/frontend/views/configuration/summary.jade
index 9a6e553..a04f0db 100644
--- a/modules/web-console/frontend/views/configuration/summary.jade
+++ b/modules/web-console/frontend/views/configuration/summary.jade
@@ -21,7 +21,7 @@ mixin hard-link(ref, txt)
 
 .docs-header
     h1 Configurations Summary
-.docs-body
+.docs-body.summary
     ignite-information
         ul
             li Preview XML configurations for #[a(href='https://apacheignite.readme.io/docs/clients-vs-servers' target='_blank') server and client] nodes
@@ -29,7 +29,6 @@ mixin hard-link(ref, txt)
             li Preview #[a(href='https://apacheignite.readme.io/docs/docker-deployment' target='_blank') Docker file]
             li Preview POM dependencies
             li Download ready-to-use Maven project
-
     hr
     .padding-dflt(ng-if='ui.ready && (!clusters || clusters.length == 0)')
         | You have no clusters configured. Please configure them #[a(ui-sref='base.configuration.clusters') here].
@@ -37,13 +36,21 @@ mixin hard-link(ref, txt)
     div(ng-show='clusters && clusters.length > 0' ignite-loading='summaryPage' ignite-loading-text='Loading summary screen...' ignite-loading-position='top')
         +main-table('clusters', 'clustersView', 'clusterName', 'selectItem(row)', '{{$index + 1}}) {{row.name}}', 'name')
         div(ng-show='selectedItem && contentVisible(displayedRows, selectedItem)')
-            .padding-top-dflt(bs-affix)
-                button.btn.btn-primary(id='download' ng-click='downloadConfiguration()' bs-tooltip='' data-title='Download project' data-placement='bottom') Download project
-                .btn.btn-primary(bs-tooltip='' data-title='Preview generated project structure' data-placement='bottom')
-                    div(bs-popover data-template-url='/configuration/summary-project-structure.html', data-placement='bottom', data-trigger='click' data-auto-close='true')
-                        i.fa.fa-sitemap
-                        label.tipLabel Project structure
-                button.btn.btn-primary(id='proprietary-jdbc-drivers' ng-if='downloadJdbcDriversVisible()' ng-click='downloadJdbcDrivers()' bs-tooltip='' data-title='Open proprietary JDBC drivers download pages' data-placement='bottom') Download JDBC drivers
+            .actions.padding-top-dflt(bs-affix)
+                div
+                    button.btn.btn-primary(id='download' ng-click='downloadConfiguration()' bs-tooltip='' data-title='Download project' data-placement='bottom' ng-disabled='isPrepareDownloading')
+                        div
+                            i.fa.fa-fw.fa-download(ng-hide='isPrepareDownloading')
+                            i.fa.fa-fw.fa-refresh.fa-spin(ng-show='isPrepareDownloading')
+                            span.tipLabel Download project
+                    button.btn.btn-primary(bs-tooltip='' data-title='Preview generated project structure' data-placement='bottom')
+                        div(bs-popover data-template-url='/configuration/summary-project-structure.html', data-placement='bottom', data-trigger='click' data-auto-close='true')
+                            i.fa.fa-sitemap
+                            label.tipLabel Project structure
+                    button.btn.btn-primary(id='proprietary-jdbc-drivers' ng-if='downloadJdbcDriversVisible()' ng-click='downloadJdbcDrivers()' bs-tooltip='' data-title='Open proprietary JDBC drivers download pages' data-placement='bottom') Download JDBC drivers
+                .actions-note(ng-show='ui.isSafari')
+                    i.icon-note
+                    label "Download project" is not fully supported in Safari. Please rename downloaded file from "Unknown" to "&lt;project-name&gt;.zip"
                 hr
             .bs-affix-fix
             .panel-group(bs-collapse ng-init='ui.activePanels=[0,1]' ng-model='ui.activePanels' data-allow-multiple='true')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/views/settings/admin.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/settings/admin.jade b/modules/web-console/frontend/views/settings/admin.jade
index 862d959..c985826 100644
--- a/modules/web-console/frontend/views/settings/admin.jade
+++ b/modules/web-console/frontend/views/settings/admin.jade
@@ -14,63 +14,38 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-.row(ng-controller='adminController')
+mixin grid-settings()
+    i.fa.fa-bars(data-animation='am-flip-x' bs-dropdown='' aria-haspopup='true' aria-expanded='expanded' data-auto-close='1' data-trigger='click')
+    ul.select.dropdown-menu(role='menu')
+        li(ng-repeat='item in ctrl.gridOptions.categories|filter:{selectable:true}')
+            a(ng-click='ctrl.toggleColumns(item, !item.visible)')
+                i.fa.fa-check-square-o.pull-left(ng-if='item.visible')
+                i.fa.fa-square-o.pull-left(ng-if='!item.visible')
+                span {{::item.name}}
+        li.divider
+        li
+            a(ng-click='ctrl.selectAllColumns()') Select all
+        li
+            a(ng-click='ctrl.clearAllColumns()') Clear all
+        li.divider
+        li
+            a(ng-click='$hide()') Close
+
+.admin-page.row(ng-controller='adminController')
     .docs-content.greedy
         .docs-header
             h1 List of registered users
             hr
         .docs-body
-            .col-xs-12
-                table.table.table-striped.table-vertical-middle.admin(st-table='displayedUsers' st-safe-src='users')
-                    thead
-                        tr
-                            th.header(colspan='10')
-                                .col-xs-3
-                                    input.form-control(type='text' st-search='label' placeholder='Filter users...')
-                                .col-xs-9.admin-summary.text-right(colspan='10')
-                                    strong Total users: {{ users.length }}
-                                .col-xs-offset-6.col-xs-6.text-right
-                                    div(st-pagination st-items-by-page='15' st-displayed-pages='5' st-template='../templates/pagination.html')
-                        tr
-                            th(st-sort='userName') User
-                            th(st-sort='email') Email
-                            th(st-sort='company') Company
-                            th(st-sort='country') Country
-                            th.col-xs-2(st-sort='lastLogin' st-sort-default='reverse') Last login
-                            th.text-nowrap(st-sort='counters.clusters' st-descending-first bs-tooltip='"Clusters count"' data-placement='top')
-                                i.fa.fa-sitemap()
-                            th.text-nowrap(st-sort='counters.models' st-descending-first bs-tooltip='"Models count"' data-placement='top')
-                                i.fa.fa-object-group()
-                            th.text-nowrap(st-sort='counters.caches' st-descending-first bs-tooltip='"Caches count"' data-placement='top')
-                                i.fa.fa-database()
-                            th.text-nowrap(st-sort='counters.igfs' st-descending-first bs-tooltip='"IGFS count"' data-placement='top')
-                                i.fa.fa-folder-o()
-                            th(width='1%') Actions
-                    tbody
-                        tr(ng-repeat='row in displayedUsers track by row._id')
-                            td {{::row.userName}}
-                            td
-                                a(ng-href='mailto:{{::row.email}}') {{::row.email}}
-                            td {{::row.company}}
-                            td {{::row.countryCode}}
-                            td {{::row.lastLogin | date:'medium'}}
-                            td {{::row.counters.clusters}}
-                            td {{::row.counters.models}}
-                            td {{::row.counters.caches}}
-                            td {{::row.counters.igfs}}
-                            td.text-center
-                                a.btn.btn-default.dropdown-toggle(bs-dropdown='' ng-show='row._id != user._id' data-placement='bottom-right')
-                                    i.fa.fa-gear &nbsp;
-                                    span.caret
-                                ul.dropdown-menu(role='menu')
-                                    li
-                                        a(ng-click='becomeUser(row)') Become this user
-                                    li
-                                        a(ng-click='toggleAdmin(row)' ng-if='row.admin && row._id !== user._id') Revoke admin
-                                        a(ng-click='toggleAdmin(row)' ng-if='!row.admin && row._id !== user._id')  Grant admin
-                                    li
-                                        a(ng-click='removeUser(row)') Remove user
-                    tfoot
-                        tr
-                            td.text-right(colspan='10')
-                                div(st-pagination st-items-by-page='15' st-displayed-pages='5' st-template='../templates/pagination.html')
+            .row
+                .col-xs-12
+                    .panel.panel-default
+                        .panel-heading.ui-grid-settings
+                            +grid-settings
+                            label Total users: 
+                                strong {{ users.length }}&nbsp;&nbsp;&nbsp;
+                            label Showing users: 
+                                strong {{ ctrl.gridApi.grid.getVisibleRows().length }}
+                                sub(ng-show='users.length === ctrl.gridApi.grid.getVisibleRows().length') all
+                        .panel-collapse
+                            .grid(ui-grid='ctrl.gridOptions' ui-grid-resize-columns ui-grid-selection ui-grid-pinning)

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/views/sql/notebook-new.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/sql/notebook-new.jade b/modules/web-console/frontend/views/sql/notebook-new.jade
index 8d9e8c4..9585e92 100644
--- a/modules/web-console/frontend/views/sql/notebook-new.jade
+++ b/modules/web-console/frontend/views/sql/notebook-new.jade
@@ -21,7 +21,7 @@
                 button.close(ng-click='$hide()') &times;
                 h4.modal-title
                     i.fa.fa-file-o
-                    | New SQL notebook
+                    | New query notebook
             form.form-horizontal.modal-body.row(name='ui.inputForm' novalidate)
                 div
                     .col-sm-2


[18/50] [abbrv] ignite git commit: ignite-4450 Need release locks for failing nodes during exchange processing.

Posted by yz...@apache.org.
ignite-4450 Need release locks for failing nodes during exchange processing.


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 41dddb87da53bbe72594b0b4bb9e2a396a57b986
Parents: c864fe4
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 22 11:26:18 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 22 11:26:18 2016 +0300

----------------------------------------------------------------------
 .../GridDhtPartitionsExchangeFuture.java        |   6 +-
 .../CacheLockReleaseNodeLeaveTest.java          | 135 +++++++++++++++++++
 2 files changed, 139 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/41dddb87/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 4f34401..2cfc0d0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -805,8 +805,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
             }
             catch (IgniteFutureTimeoutCheckedException ignored) {
                 if (dumpedObjects < DUMP_PENDING_OBJECTS_THRESHOLD) {
-                    U.warn(log, "Failed to wait for locks release future. " +
-                        "Dumping pending objects that might be the cause: " + cctx.localNodeId());
+                    U.warn(log, "Failed to wait for locks release future. Dumping pending objects that might be the " +
+                        "cause [topVer=" + topologyVersion() + ", nodeId=" + cctx.localNodeId() + "]: ");
 
                     U.warn(log, "Locked keys:");
 
@@ -1556,6 +1556,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
         if (isDone() || !enterBusy())
             return;
 
+        cctx.mvcc().removeExplicitNodeLocks(node.id(), topologyVersion());
+
         try {
             onDiscoveryEvent(new IgniteRunnable() {
                 @Override public void run() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/41dddb87/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
index e84fd3f..969b991 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
@@ -20,9 +20,13 @@ package org.apache.ignite.internal.processors.cache.distributed;
 import java.util.concurrent.Callable;
 import java.util.concurrent.locks.Lock;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 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;
@@ -158,4 +162,135 @@ public class CacheLockReleaseNodeLeaveTest extends GridCommonAbstractTest {
 
         fut2.get(5, SECONDS);
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLockRelease2() throws Exception {
+        final Ignite ignite0 = startGrid(0);
+
+        Ignite ignite1 = startGrid(1);
+
+        Lock lock = ignite1.cache(null).lock("key");
+        lock.lock();
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                startGrid(2);
+
+                return null;
+            }
+        });
+
+        final AffinityTopologyVersion topVer = new AffinityTopologyVersion(2, 0);
+
+        // Wait when affinity change exchange start.
+        boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                AffinityTopologyVersion topVer0 =
+                    ((IgniteKernal)ignite0).context().cache().context().exchange().topologyVersion();
+
+                return topVer.compareTo(topVer0) < 0;
+            }
+        }, 10_000);
+
+        assertTrue(wait);
+
+        assertFalse(fut.isDone());
+
+        ignite1.close();
+
+        fut.get(10_000);
+
+        Ignite ignite2 = ignite(2);
+
+        lock = ignite2.cache(null).lock("key");
+        lock.lock();
+        lock.unlock();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLockRelease3() throws Exception {
+        startGrid(0);
+
+        Ignite ignite1 = startGrid(1);
+
+        awaitPartitionMapExchange();
+
+        Lock lock = ignite1.cache(null).lock("key");
+        lock.lock();
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                startGrid(2);
+
+                return null;
+            }
+        });
+
+        assertFalse(fut.isDone());
+
+        ignite1.close();
+
+        fut.get(10_000);
+
+        Ignite ignite2 = ignite(2);
+
+        lock = ignite2.cache(null).lock("key");
+        lock.lock();
+        lock.unlock();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxLockRelease2() throws Exception {
+        final Ignite ignite0 = startGrid(0);
+
+        Ignite ignite1 = startGrid(1);
+
+        IgniteCache cache = ignite1.cache(null);
+        ignite1.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+        cache.get(1);
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                startGrid(2);
+
+                return null;
+            }
+        });
+
+        final AffinityTopologyVersion topVer = new AffinityTopologyVersion(2, 0);
+
+        // Wait when affinity change exchange start.
+        boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                AffinityTopologyVersion topVer0 =
+                    ((IgniteKernal)ignite0).context().cache().context().exchange().topologyVersion();
+
+                return topVer.compareTo(topVer0) < 0;
+            }
+        }, 10_000);
+
+        assertTrue(wait);
+
+        assertFalse(fut.isDone());
+
+        ignite1.close();
+
+        fut.get(10_000);
+
+        Ignite ignite2 = ignite(2);
+
+        cache = ignite2.cache(null);
+
+        try (Transaction tx = ignite2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+            cache.get(1);
+
+            tx.commit();
+        }
+    }
 }


[04/50] [abbrv] ignite git commit: Null argument check for cache.putAll.

Posted by yz...@apache.org.
Null argument check for cache.putAll.

(cherry picked from commit c0f1a21)


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

Branch: refs/heads/ignite-comm-balance-master
Commit: e82eefe7a7d7f541e6d2fc4b166843b87ec9ef87
Parents: 09c634e
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 20 12:35:51 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 20 13:21:28 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/cache/GridCacheAdapter.java  | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e82eefe7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 8187e8f..f235f6a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -2661,6 +2661,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public void putAll(@Nullable final Map<? extends K, ? extends V> m) throws IgniteCheckedException {
+        A.notNull(m, "map");
+
         if (F.isEmpty(m))
             return;
 


[13/50] [abbrv] ignite git commit: IGNITE-4469: Hadoop: set default value of offheap memory page and shuffle messages to 1Mb.

Posted by yz...@apache.org.
IGNITE-4469: Hadoop: set default value of offheap memory page and shuffle messages to 1Mb.


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 854d1a5d9ccf4d24bd3cdcbb1a60a52c78ef5c88
Parents: 598b464
Author: devozerov <vo...@gridgain.com>
Authored: Wed Dec 21 13:08:30 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Dec 21 13:08:30 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/hadoop/HadoopJobProperty.java    | 4 ++--
 .../internal/processors/hadoop/shuffle/HadoopShuffleJob.java    | 2 +-
 .../hadoop/shuffle/collections/HadoopMultimapBase.java          | 5 ++++-
 3 files changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/854d1a5d/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
index 9dd430b..4398acd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
@@ -71,7 +71,7 @@ public enum HadoopJobProperty {
     /**
      * Size in bytes of single memory page which will be allocated for data structures in shuffle.
      * <p>
-     * By default is {@code 32 * 1024}.
+     * Defaults to {@code 1Mb}.
      */
     SHUFFLE_OFFHEAP_PAGE_SIZE("ignite.shuffle.offheap.page.size"),
 
@@ -98,7 +98,7 @@ public enum HadoopJobProperty {
     /**
      * Defines approximate size in bytes of shuffle message which will be passed over wire from mapper to reducer.
      * <p>
-     * Defaults to 128Kb.
+     * Defaults to {@code 1Mb}.
      */
     SHUFFLE_MSG_SIZE("ignite.shuffle.message.size"),
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/854d1a5d/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
index 214a335..1c546a1 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
@@ -76,7 +76,7 @@ import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.get
  */
 public class HadoopShuffleJob<T> implements AutoCloseable {
     /** */
-    private static final int DFLT_SHUFFLE_MSG_SIZE = 128 * 1024;
+    private static final int DFLT_SHUFFLE_MSG_SIZE = 1024 * 1024;
 
     /** */
     private final HadoopJob job;

http://git-wip-us.apache.org/repos/asf/ignite/blob/854d1a5d/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java
index 39b7c51..d7a4541 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java
@@ -40,6 +40,9 @@ import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.get
  * Base class for all multimaps.
  */
 public abstract class HadoopMultimapBase implements HadoopMultimap {
+    /** Default offheap page size. */
+    private static final int DFLT_OFFHEAP_PAGE_SIZE = 1024 * 1024;
+
     /** */
     protected final GridUnsafeMemory mem;
 
@@ -59,7 +62,7 @@ public abstract class HadoopMultimapBase implements HadoopMultimap {
 
         this.mem = mem;
 
-        pageSize = get(jobInfo, SHUFFLE_OFFHEAP_PAGE_SIZE, 32 * 1024);
+        pageSize = get(jobInfo, SHUFFLE_OFFHEAP_PAGE_SIZE, DFLT_OFFHEAP_PAGE_SIZE);
     }
 
     /**


[21/50] [abbrv] ignite git commit: IGNITE-4439 - Attribute based node filter

Posted by yz...@apache.org.
IGNITE-4439 - Attribute based node filter


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 2591c160efc4251cb33854955970c93ec20d6b24
Parents: 7094c0f
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Thu Dec 22 13:05:35 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Thu Dec 22 13:05:35 2016 -0800

----------------------------------------------------------------------
 .../apache/ignite/util/AttributeNodeFilter.java | 105 +++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   3 +
 .../util/AttributeNodeFilterSelfTest.java       | 184 +++++++++++++++++++
 3 files changed, 292 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2591c160/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java b/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java
new file mode 100644
index 0000000..e2b972b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java
@@ -0,0 +1,105 @@
+/*
+ * 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.util;
+
+import java.util.Collections;
+import java.util.Map;
+import org.apache.ignite.cluster.ClusterGroup;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.services.ServiceConfiguration;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Implementation of {@code IgnitePredicate<ClusterNode>} based on
+ * {@link IgniteConfiguration#getUserAttributes() user attributes}.
+ * This filter can be used in methods like {@link ClusterGroup#forPredicate(IgnitePredicate)},
+ * {@link CacheConfiguration#setNodeFilter(IgnitePredicate)},
+ * {@link ServiceConfiguration#setNodeFilter(IgnitePredicate)}, etc.
+ * <p>
+ * The filter will evaluate to true if a node has <b>all</b> provided attributes set to
+ * corresponding values. Here is an example of how you can configure node filter for a
+ * cache or a service so that it's deployed only on nodes that have {@code group}
+ * attribute set to value {@code data}:
+ * <pre name="code" class="xml">
+ * &lt;property name=&quot;nodeFilter&quot;&gt;
+ *     &lt;bean class=&quot;org.apache.ignite.util.ClusterAttributeNodeFilter&quot;&gt;
+ *         &lt;constructor-arg value="group"/&gt;
+ *         &lt;constructor-arg value="data"/&gt;
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * </pre>
+ * You can also specify multiple attributes for the filter:
+ * <pre name="code" class="xml">
+ * &lt;property name=&quot;nodeFilter&quot;&gt;
+ *     &lt;bean class=&quot;org.apache.ignite.util.ClusterAttributeNodeFilter&quot;&gt;
+ *         &lt;constructor-arg&gt;
+ *             &lt;map&gt;
+ *                 &lt;entry key=&quot;cpu-group&quot; value=&quot;high&quot;/&gt;
+ *                 &lt;entry key=&quot;memory-group&quot; value=&quot;high&quot;/&gt;
+ *             &lt;/map&gt;
+ *         &lt;/constructor-arg&gt;
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * </pre>
+ * With this configuration a cache or a service will deploy only on nodes that have both
+ * {@code cpu-group} and {@code memory-group} attributes set to value {@code high}.
+ */
+public class AttributeNodeFilter implements IgnitePredicate<ClusterNode> {
+    /** Attributes. */
+    private final Map<String, Object> attrs;
+
+    /**
+     * Creates new node filter with a single attribute value.
+     *
+     * @param attrName Attribute name.
+     * @param attrVal Attribute value.
+     */
+    public AttributeNodeFilter(String attrName, @Nullable Object attrVal) {
+        A.notNull(attrName, "attrName");
+
+        attrs = Collections.singletonMap(attrName, attrVal);
+    }
+
+    /**
+     * Creates new node filter with a set of attributes.
+     *
+     * @param attrs Attributes.
+     */
+    public AttributeNodeFilter(Map<String, Object> attrs) {
+        A.notNull(attrs, "attrs");
+
+        this.attrs = attrs;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean apply(ClusterNode node) {
+        Map<String, Object> nodeAttrs = node.attributes();
+
+        for (Map.Entry<String, Object> attr : attrs.entrySet()) {
+            if (!F.eq(nodeAttrs.get(attr.getKey()), attr.getValue()))
+                return false;
+        }
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/2591c160/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index c6281df..8ccec34 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -20,6 +20,7 @@ package org.apache.ignite.testsuites;
 import java.util.Set;
 import junit.framework.TestSuite;
 import org.apache.ignite.GridSuppressedExceptionSelfTest;
+import org.apache.ignite.util.AttributeNodeFilterSelfTest;
 import org.apache.ignite.internal.ClusterGroupHostsSelfTest;
 import org.apache.ignite.internal.ClusterGroupSelfTest;
 import org.apache.ignite.internal.GridFailFastNodeFailureDetectionSelfTest;
@@ -149,6 +150,8 @@ public class IgniteBasicTestSuite extends TestSuite {
 
         suite.addTestSuite(SecurityPermissionSetBuilderTest.class);
 
+        suite.addTestSuite(AttributeNodeFilterSelfTest.class);
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/2591c160/modules/core/src/test/java/org/apache/ignite/util/AttributeNodeFilterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/util/AttributeNodeFilterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/util/AttributeNodeFilterSelfTest.java
new file mode 100644
index 0000000..ac3800f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/util/AttributeNodeFilterSelfTest.java
@@ -0,0 +1,184 @@
+/*
+ * 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.util;
+
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.util.Collections;
+import java.util.Map;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Tests for {@link AttributeNodeFilter}.
+ */
+public class AttributeNodeFilterSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private Map<String, ?> attrs;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER));
+
+        if (attrs != null)
+            cfg.setUserAttributes(attrs);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        attrs = null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSingleAttribute() throws Exception {
+        IgnitePredicate<ClusterNode> filter = new AttributeNodeFilter("attr", "value");
+
+        assertTrue(filter.apply(nodeProxy(F.asMap("attr", "value"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr", "wrong"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr", null))));
+        assertFalse(filter.apply(nodeProxy(Collections.<String, Object>emptyMap())));
+        assertFalse(filter.apply(nodeProxy(F.asMap("wrong", "value"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("null", "value"))));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSingleAttributeNullValue() throws Exception {
+        IgnitePredicate<ClusterNode> filter = new AttributeNodeFilter("attr", null);
+
+        assertTrue(filter.apply(nodeProxy(F.asMap("attr", null))));
+        assertTrue(filter.apply(nodeProxy(Collections.<String, Object>emptyMap())));
+        assertTrue(filter.apply(nodeProxy(F.asMap("wrong", "value"))));
+        assertTrue(filter.apply(nodeProxy(F.asMap("wrong", null))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr", "value"))));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMultipleAttributes() throws Exception {
+        IgnitePredicate<ClusterNode> filter =
+            new AttributeNodeFilter(F.<String, Object>asMap("attr1", "value1", "attr2", "value2"));
+
+        assertTrue(filter.apply(nodeProxy(F.asMap("attr1", "value1", "attr2", "value2"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr1", "wrong", "attr2", "value2"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr1", "value1", "attr2", "wrong"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr1", "wrong", "attr2", "wrong"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr1", "value1"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr2", "value2"))));
+        assertFalse(filter.apply(nodeProxy(Collections.<String, Object>emptyMap())));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMultipleAttributesNullValues() throws Exception {
+        IgnitePredicate<ClusterNode> filter = new AttributeNodeFilter(F.asMap("attr1", null, "attr2", null));
+
+        assertTrue(filter.apply(nodeProxy(F.asMap("attr1", null, "attr2", null))));
+        assertTrue(filter.apply(nodeProxy(F.asMap("attr1", null))));
+        assertTrue(filter.apply(nodeProxy(F.asMap("attr2", null))));
+        assertTrue(filter.apply(nodeProxy(Collections.<String, Object>emptyMap())));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr1", "value1"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr2", "value2"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr1", "value1", "attr2", "value2"))));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClusterGroup() throws Exception {
+        Ignite group1 = startGridsMultiThreaded(3);
+
+        attrs = F.asMap("group", "data");
+
+        Ignite group2 = startGridsMultiThreaded(3, 2);
+
+        assertEquals(2, group1.cluster().forPredicate(new AttributeNodeFilter("group", "data")).nodes().size());
+        assertEquals(2, group2.cluster().forPredicate(new AttributeNodeFilter("group", "data")).nodes().size());
+
+        assertEquals(3, group1.cluster().forPredicate(new AttributeNodeFilter("group", null)).nodes().size());
+        assertEquals(3, group2.cluster().forPredicate(new AttributeNodeFilter("group", null)).nodes().size());
+
+        assertEquals(0, group1.cluster().forPredicate(new AttributeNodeFilter("group", "wrong")).nodes().size());
+        assertEquals(0, group2.cluster().forPredicate(new AttributeNodeFilter("group", "wrong")).nodes().size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCacheFilter() throws Exception {
+        Ignite group1 = startGridsMultiThreaded(3);
+
+        attrs = F.asMap("group", "data");
+
+        Ignite group2 = startGridsMultiThreaded(3, 2);
+
+        group1.createCache(new CacheConfiguration<>("test-cache").
+            setNodeFilter(new AttributeNodeFilter("group", "data")));
+
+        assertEquals(2, group1.cluster().forDataNodes("test-cache").nodes().size());
+        assertEquals(2, group2.cluster().forDataNodes("test-cache").nodes().size());
+
+        assertEquals(0, group1.cluster().forDataNodes("wrong").nodes().size());
+        assertEquals(0, group2.cluster().forDataNodes("wrong").nodes().size());
+    }
+
+    /**
+     * @param attrs Attributes.
+     * @return Node proxy.
+     */
+    private static ClusterNode nodeProxy(final Map<String, ?> attrs) {
+        return (ClusterNode)Proxy.newProxyInstance(
+            ClusterNode.class.getClassLoader(),
+            new Class[] { ClusterNode.class },
+            new InvocationHandler() {
+                @SuppressWarnings("SuspiciousMethodCalls")
+                @Override public Object invoke(Object proxy, Method mtd, Object[] args) throws Throwable {
+                    if ("attributes".equals(mtd.getName()))
+                        return attrs;
+
+                    throw new UnsupportedOperationException();
+                }
+            });
+    }
+}


[05/50] [abbrv] ignite git commit: Fixed 'singleRmv' flag for cache.remove(k, v) operation in tx cache.

Posted by yz...@apache.org.
Fixed 'singleRmv' flag for cache.remove(k, v) operation in tx cache.


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 126ab60fe6fa0f47e19a26dafecc7feb7c57b60b
Parents: c0f1a21
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 20 15:15:04 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 20 15:15:04 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/cache/GridCacheAdapter.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/126ab60f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 50869be..74d1553 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -2951,6 +2951,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /**
      * @param key Key.
+     * @param filter Filter.
      * @return {@code True} if entry was removed.
      * @throws IgniteCheckedException If failed.
      */
@@ -2962,7 +2963,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                     Collections.singletonList(key),
                     /*retval*/false,
                     filter,
-                    /*singleRmv*/true).get().success();
+                    /*singleRmv*/filter == null).get().success();
             }
 
             @Override public String toString() {


[32/50] [abbrv] ignite git commit: Web console beta-7.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/Custom.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Custom.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Custom.service.js
new file mode 100644
index 0000000..a185485
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/configuration/generator/Custom.service.js
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+// Optional content generation entry point.
+export default class IgniteCustomGenerator {
+    optionalContent(zip, cluster) { // eslint-disable-line no-unused-vars
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js
index f9776a2..bcfa2e2 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js
@@ -18,7 +18,7 @@
 /**
  * Docker file generation entry point.
  */
-class GeneratorDocker {
+export default class IgniteDockerGenerator {
     /**
      * Generate from section.
      *
@@ -74,5 +74,3 @@ class GeneratorDocker {
         ].join('\n');
     }
 }
-
-export default ['GeneratorDocker', GeneratorDocker];


[49/50] [abbrv] ignite git commit: cheats

Posted by yz...@apache.org.
cheats

* 1. add partition mapping to near upd resp
3. avoid topology lock
4. avoid contention on version generation
* 6. org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache#unlockEntries
* 7. org.apache.ignite.internal.processors.cache.GridCacheEvictionManager#touch(org.apache.ignite.internal.processors.cache.GridCacheEntryEx, org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion)
8. org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture#mapOnTopology - cache.topology().readLock();
*9. org.apache.ignite.internal.processors.cache.GridCacheUtils#affinityNodes(org.apache.ignite.internal.processors.cache.GridCacheContext, org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion)
*10. org.apache.ignite.internal.managers.discovery.GridDiscoveryManager.DiscoCache#cacheAffinityNodes - change to cache id
* NOT FULLY - 11. org.apache.ignite.internal.processors.cache.version.GridCacheVersionManager#next(long, boolean, boolean, byte)
12. org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopologyImpl#nodes(int, org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion) - try replace with array for benchmark

postpone
2. avoid double context switch
5. put atomic futures per partition


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 0706f1afff2febb0546e403dec46329336e03ba7
Parents: 855399a
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Dec 28 16:48:52 2016 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Dec 28 16:48:52 2016 +0300

----------------------------------------------------------------------
 .../ignite/codegen/MessageCodeGenerator.java    |   2 +-
 .../discovery/GridDiscoveryManager.java         | 257 +++++--------------
 .../processors/cache/GridCacheUtils.java        |   6 +-
 .../cache/affinity/GridCacheAffinityImpl.java   |   2 +-
 .../dht/GridClientPartitionTopology.java        |  14 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   4 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |  17 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  25 +-
 .../GridNearAtomicSingleUpdateFuture.java       |  78 +++---
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  79 +++---
 .../atomic/GridNearAtomicUpdateResponse.java    |  35 ++-
 .../cache/version/GridCacheVersion.java         |   2 +-
 .../cache/version/GridCacheVersionManager.java  |  54 +++-
 .../clock/GridClockSyncProcessor.java           |  54 ++--
 .../GridDiscoveryManagerAliveCacheSelfTest.java |  58 +----
 .../discovery/GridDiscoveryManagerSelfTest.java |   2 +-
 16 files changed, 294 insertions(+), 395 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0706f1af/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
----------------------------------------------------------------------
diff --git a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
index 65ebd8d..5a26187 100644
--- a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
+++ b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
@@ -924,4 +924,4 @@ public class MessageCodeGenerator {
             }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0706f1af/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 9aa4db1..9ecd78e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.managers.discovery;
 
-import java.io.Externalizable;
 import java.io.Serializable;
 import java.lang.management.GarbageCollectorMXBean;
 import java.lang.management.ManagementFactory;
@@ -144,9 +143,6 @@ import static org.apache.ignite.plugin.segmentation.SegmentationPolicy.NOOP;
  * Discovery SPI manager.
  */
 public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
-    /** Fake key for {@code null}-named caches. Used inside {@link DiscoCache}. */
-    private static final String NULL_CACHE_NAME = UUID.randomUUID().toString();
-
     /** Metrics update frequency. */
     private static final long METRICS_UPDATE_FREQ = 3000;
 
@@ -1577,7 +1573,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return Collection of cache nodes.
      */
     public Collection<ClusterNode> nodes(AffinityTopologyVersion topVer) {
-        return resolveDiscoCache(null, topVer).allNodes();
+        return resolveDiscoCache(topVer).allNodes();
     }
 
     /**
@@ -1585,7 +1581,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return All server nodes for given topology version.
      */
     public List<ClusterNode> serverNodes(AffinityTopologyVersion topVer) {
-        return resolveDiscoCache(null, topVer).srvNodes;
+        return resolveDiscoCache(topVer).srvNodes;
     }
 
     /**
@@ -1596,7 +1592,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return Node.
      */
     public ClusterNode node(AffinityTopologyVersion topVer, UUID id) {
-        return resolveDiscoCache(null, topVer).node(id);
+        return resolveDiscoCache(topVer).node(id);
     }
 
     /**
@@ -1607,49 +1603,38 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return Collection of cache nodes.
      */
     public Collection<ClusterNode> cacheNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) {
-        return resolveDiscoCache(cacheName, topVer).cacheNodes(cacheName, topVer.topologyVersion());
+        return resolveDiscoCache(topVer).cacheNodes(cacheName, topVer.topologyVersion());
     }
 
     /**
-     * Gets all nodes with at least one cache configured.
+     * Gets cache nodes for cache with given ID.
      *
+     * @param cacheId Cache ID.
      * @param topVer Topology version.
      * @return Collection of cache nodes.
      */
-    public Collection<ClusterNode> cacheNodes(AffinityTopologyVersion topVer) {
-        return resolveDiscoCache(null, topVer).allNodesWithCaches(topVer.topologyVersion());
+    public Collection<ClusterNode> cacheNodes(int cacheId, AffinityTopologyVersion topVer) {
+        return resolveDiscoCache(topVer).cacheNodes(cacheId, topVer.topologyVersion());
     }
 
     /**
-     * Gets cache remote nodes for cache with given name.
-     *
-     * @param topVer Topology version.
-     * @return Collection of cache nodes.
-     */
-    public Collection<ClusterNode> remoteCacheNodes(AffinityTopologyVersion topVer) {
-        return resolveDiscoCache(null, topVer).remoteCacheNodes(topVer.topologyVersion());
-    }
-
-    /**
-     * Gets cache nodes for cache with given name.
+     * Gets all nodes with at least one cache configured.
      *
-     * @param cacheName Cache name.
      * @param topVer Topology version.
      * @return Collection of cache nodes.
      */
-    Collection<ClusterNode> aliveCacheNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) {
-        return resolveDiscoCache(cacheName, topVer).aliveCacheNodes(cacheName, topVer.topologyVersion());
+    public Collection<ClusterNode> cacheNodes(AffinityTopologyVersion topVer) {
+        return resolveDiscoCache(topVer).allNodesWithCaches(topVer.topologyVersion());
     }
 
     /**
      * Gets cache remote nodes for cache with given name.
      *
-     * @param cacheName Cache name.
      * @param topVer Topology version.
      * @return Collection of cache nodes.
      */
-    Collection<ClusterNode> aliveRemoteCacheNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) {
-        return resolveDiscoCache(cacheName, topVer).aliveRemoteCacheNodes(cacheName, topVer.topologyVersion());
+    public Collection<ClusterNode> remoteCacheNodes(AffinityTopologyVersion topVer) {
+        return resolveDiscoCache(topVer).remoteCacheNodes(topVer.topologyVersion());
     }
 
     /**
@@ -1657,7 +1642,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return Oldest alive server nodes with at least one cache configured.
      */
     @Nullable public ClusterNode oldestAliveCacheServerNode(AffinityTopologyVersion topVer) {
-        DiscoCache cache = resolveDiscoCache(null, topVer);
+        DiscoCache cache = resolveDiscoCache(topVer);
 
         Map.Entry<ClusterNode, Boolean> e = cache.aliveSrvNodesWithCaches.firstEntry();
 
@@ -1672,7 +1657,18 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return Collection of cache affinity nodes.
      */
     public Collection<ClusterNode> cacheAffinityNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) {
-        return resolveDiscoCache(cacheName, topVer).cacheAffinityNodes(cacheName, topVer.topologyVersion());
+        return resolveDiscoCache(topVer).cacheAffinityNodes(CU.cacheId(cacheName), topVer.topologyVersion());
+    }
+
+    /**
+     * Gets cache nodes for cache with given ID that participate in affinity calculation.
+     *
+     * @param cacheId Cache ID.
+     * @param topVer Topology version.
+     * @return Collection of cache affinity nodes.
+     */
+    public Collection<ClusterNode> cacheAffinityNodes(int cacheId, AffinityTopologyVersion topVer) {
+        return resolveDiscoCache(topVer).cacheAffinityNodes(cacheId, topVer.topologyVersion());
     }
 
     /**
@@ -1749,25 +1745,34 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return {@code True} if cache with given name has at least one node with near cache enabled.
      */
     public boolean hasNearCache(@Nullable String cacheName, AffinityTopologyVersion topVer) {
-        return resolveDiscoCache(cacheName, topVer).hasNearCache(cacheName);
+        return resolveDiscoCache(topVer).hasNearCache(CU.cacheId(cacheName));
+    }
+
+    /**
+     * Checks if cache with given ID has at least one node with near cache enabled.
+     *
+     * @param cacheId Cache ID.
+     * @param topVer Topology version.
+     * @return {@code True} if cache with given name has at least one node with near cache enabled.
+     */
+    public boolean hasNearCache(int cacheId, AffinityTopologyVersion topVer) {
+        return resolveDiscoCache(topVer).hasNearCache(cacheId);
     }
 
     /**
      * Gets discovery cache for given topology version.
      *
-     * @param cacheName Cache name (participates in exception message).
      * @param topVer Topology version.
      * @return Discovery cache.
      */
-    private DiscoCache resolveDiscoCache(@Nullable String cacheName, AffinityTopologyVersion topVer) {
+    private DiscoCache resolveDiscoCache(AffinityTopologyVersion topVer) {
         Snapshot snap = topSnap.get();
 
         DiscoCache cache = AffinityTopologyVersion.NONE.equals(topVer) || topVer.equals(snap.topVer) ?
             snap.discoCache : discoCacheHist.get(topVer);
 
         if (cache == null) {
-            throw new IgniteException("Failed to resolve nodes topology [cacheName=" + cacheName +
-                ", topVer=" + topVer +
+            throw new IgniteException("Failed to resolve nodes topology [topVer=" + topVer +
                 ", history=" + discoCacheHist.keySet() +
                 ", snap=" + snap +
                 ", locNode=" + ctx.discovery().localNode() + ']');
@@ -2093,19 +2098,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             evts.add(new GridTuple5<>(type, topVer, node, topSnapshot, data));
         }
 
-        /**
-         * @param node Node to get a short description for.
-         * @return Short description for the node to be used in 'quiet' mode.
-         */
-        private String quietNode(ClusterNode node) {
-            assert node != null;
-
-            return "nodeId8=" + node.id().toString().substring(0, 8) + ", " +
-                "addrs=" + U.addressesAsString(node) + ", " +
-                "order=" + node.order() + ", " +
-                "CPUs=" + node.metrics().getTotalCpus();
-        }
-
         /** {@inheritDoc} */
         @Override protected void body() throws InterruptedException {
             while (!isCancelled()) {
@@ -2415,11 +2407,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         /** Topology await version. */
         private long awaitVer;
 
-        /** Empty constructor required by {@link Externalizable}. */
-        private DiscoTopologyFuture() {
-            // No-op.
-        }
-
         /**
          * @param ctx Context.
          * @param awaitVer Await version.
@@ -2509,19 +2496,15 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
         /** Cache nodes by cache name. */
         @GridToStringInclude
-        private final Map<String, Collection<ClusterNode>> allCacheNodes;
-
-        /** Remote cache nodes by cache name. */
-        @GridToStringInclude
-        private final Map<String, Collection<ClusterNode>> rmtCacheNodes;
+        private final Map<Integer, Collection<ClusterNode>> allCacheNodes;
 
         /** Cache nodes by cache name. */
         @GridToStringInclude
-        private final Map<String, Collection<ClusterNode>> affCacheNodes;
+        private final Map<Integer, Collection<ClusterNode>> affCacheNodes;
 
         /** Caches where at least one node has near cache enabled. */
         @GridToStringInclude
-        private final Set<String> nearEnabledCaches;
+        private final Set<Integer> nearEnabledCaches;
 
         /** Nodes grouped by version. */
         private final NavigableMap<IgniteProductVersion, Collection<ClusterNode>> nodesByVer;
@@ -2539,18 +2522,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         private final long maxOrder;
 
         /**
-         * Cached alive nodes list. As long as this collection doesn't accept {@code null}s use {@link
-         * #maskNull(String)} before passing raw cache names to it.
-         */
-        private final ConcurrentMap<String, Collection<ClusterNode>> aliveCacheNodes;
-
-        /**
-         * Cached alive remote nodes list. As long as this collection doesn't accept {@code null}s use {@link
-         * #maskNull(String)} before passing raw cache names to it.
-         */
-        private final ConcurrentMap<String, Collection<ClusterNode>> aliveRmtCacheNodes;
-
-        /**
          * Cached alive server remote nodes with caches.
          */
         private final ConcurrentSkipListMap<ClusterNode, Boolean> aliveSrvNodesWithCaches;
@@ -2578,20 +2549,17 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
             allNodes = Collections.unmodifiableList(all);
 
-            Map<String, Collection<ClusterNode>> cacheMap = new HashMap<>(allNodes.size(), 1.0f);
-            Map<String, Collection<ClusterNode>> rmtCacheMap = new HashMap<>(allNodes.size(), 1.0f);
-            Map<String, Collection<ClusterNode>> dhtNodesMap = new HashMap<>(allNodes.size(), 1.0f);
-            Collection<ClusterNode> nodesWithCaches = new HashSet<>(allNodes.size());
-            Collection<ClusterNode> rmtNodesWithCaches = new HashSet<>(allNodes.size());
+            Map<Integer, Collection<ClusterNode>> cacheMap = U.newHashMap(allNodes.size());
+            Map<Integer, Collection<ClusterNode>> dhtNodesMap = U.newHashMap(allNodes.size());
+            Collection<ClusterNode> nodesWithCaches = U.newHashSet(allNodes.size());
+            Collection<ClusterNode> rmtNodesWithCaches = U.newHashSet(allNodes.size());
 
-            aliveCacheNodes = new ConcurrentHashMap8<>(allNodes.size(), 1.0f);
-            aliveRmtCacheNodes = new ConcurrentHashMap8<>(allNodes.size(), 1.0f);
             aliveSrvNodesWithCaches = new ConcurrentSkipListMap<>(GridNodeOrderComparator.INSTANCE);
             nodesByVer = new TreeMap<>();
 
             long maxOrder0 = 0;
 
-            Set<String> nearEnabledSet = new HashSet<>();
+            Set<Integer> nearEnabledSet = new HashSet<>();
 
             List<ClusterNode> srvNodes = new ArrayList<>();
 
@@ -2620,21 +2588,11 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
                         addToMap(cacheMap, cacheName, node);
 
-                        if (alive(node.id()))
-                            addToMap(aliveCacheNodes, maskNull(cacheName), node);
-
                         if (filter.dataNode(node))
                             addToMap(dhtNodesMap, cacheName, node);
 
                         if (filter.nearNode(node))
-                            nearEnabledSet.add(cacheName);
-
-                        if (!loc.id().equals(node.id())) {
-                            addToMap(rmtCacheMap, cacheName, node);
-
-                            if (alive(node.id()))
-                                addToMap(aliveRmtCacheNodes, maskNull(cacheName), node);
-                        }
+                            nearEnabledSet.add(CU.cacheId(cacheName));
 
                         hasCaches = true;
                     }
@@ -2674,7 +2632,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             maxOrder = maxOrder0;
 
             allCacheNodes = Collections.unmodifiableMap(cacheMap);
-            rmtCacheNodes = Collections.unmodifiableMap(rmtCacheMap);
             affCacheNodes = Collections.unmodifiableMap(dhtNodesMap);
             allNodesWithCaches = Collections.unmodifiableCollection(nodesWithCaches);
             this.rmtNodesWithCaches = Collections.unmodifiableCollection(rmtNodesWithCaches);
@@ -2684,7 +2641,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             daemonNodes = Collections.unmodifiableList(new ArrayList<>(
                 F.view(F.concat(false, loc, rmts), F0.not(FILTER_DAEMON))));
 
-            Map<UUID, ClusterNode> nodeMap = new HashMap<>(allNodes().size() + daemonNodes.size(), 1.0f);
+            Map<UUID, ClusterNode> nodeMap = U.newHashMap(allNodes().size() + daemonNodes.size());
 
             for (ClusterNode n : F.concat(false, allNodes(), daemonNodes()))
                 nodeMap.put(n.id(), n);
@@ -2699,13 +2656,13 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
          * @param cacheName Cache name.
          * @param rich Node to add
          */
-        private void addToMap(Map<String, Collection<ClusterNode>> cacheMap, String cacheName, ClusterNode rich) {
-            Collection<ClusterNode> cacheNodes = cacheMap.get(cacheName);
+        private void addToMap(Map<Integer, Collection<ClusterNode>> cacheMap, String cacheName, ClusterNode rich) {
+            Collection<ClusterNode> cacheNodes = cacheMap.get(CU.cacheId(cacheName));
 
             if (cacheNodes == null) {
                 cacheNodes = new ArrayList<>(allNodes.size());
 
-                cacheMap.put(cacheName, cacheNodes);
+                cacheMap.put(CU.cacheId(cacheName), cacheNodes);
             }
 
             cacheNodes.add(rich);
@@ -2727,28 +2684,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         }
 
         /**
-         * Gets collection of nodes which have version equal or greater than {@code ver}.
-         *
-         * @param ver Version to check.
-         * @return Collection of nodes with version equal or greater than {@code ver}.
-         */
-        Collection<ClusterNode> elderNodes(IgniteProductVersion ver) {
-            Map.Entry<IgniteProductVersion, Collection<ClusterNode>> entry = nodesByVer.ceilingEntry(ver);
-
-            if (entry == null)
-                return Collections.emptyList();
-
-            return entry.getValue();
-        }
-
-        /**
-         * @return Versions map.
-         */
-        NavigableMap<IgniteProductVersion, Collection<ClusterNode>> versionsMap() {
-            return nodesByVer;
-        }
-
-        /**
          * Gets collection of nodes with at least one cache configured.
          *
          * @param topVer Topology version (maximum allowed node order).
@@ -2766,61 +2701,50 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
          * @return Collection of nodes.
          */
         Collection<ClusterNode> cacheNodes(@Nullable String cacheName, final long topVer) {
-            return filter(topVer, allCacheNodes.get(cacheName));
+            return filter(topVer, allCacheNodes.get(CU.cacheId(cacheName)));
         }
 
         /**
-         * Gets all remote nodes that have at least one cache configured.
+         * Gets all nodes that have cache with given ID.
          *
+         * @param cacheId Cache ID.
          * @param topVer Topology version.
          * @return Collection of nodes.
          */
-        Collection<ClusterNode> remoteCacheNodes(final long topVer) {
-            return filter(topVer, rmtNodesWithCaches);
+        Collection<ClusterNode> cacheNodes(Integer cacheId, final long topVer) {
+            return filter(topVer, allCacheNodes.get(cacheId));
         }
 
         /**
-         * Gets all nodes that have cache with given name and should participate in affinity calculation. With
-         * partitioned cache nodes with near-only cache do not participate in affinity node calculation.
-         *
-         * @param cacheName Cache name.
-         * @param topVer Topology version.
-         * @return Collection of nodes.
-         */
-        Collection<ClusterNode> cacheAffinityNodes(@Nullable String cacheName, final long topVer) {
-            return filter(topVer, affCacheNodes.get(cacheName));
-        }
-
-        /**
-         * Gets all alive nodes that have cache with given name.
+         * Gets all remote nodes that have at least one cache configured.
          *
-         * @param cacheName Cache name.
          * @param topVer Topology version.
          * @return Collection of nodes.
          */
-        Collection<ClusterNode> aliveCacheNodes(@Nullable String cacheName, final long topVer) {
-            return filter(topVer, aliveCacheNodes.get(maskNull(cacheName)));
+        Collection<ClusterNode> remoteCacheNodes(final long topVer) {
+            return filter(topVer, rmtNodesWithCaches);
         }
 
         /**
-         * Gets all alive remote nodes that have cache with given name.
+         * Gets all nodes that have cache with given ID and should participate in affinity calculation. With
+         * partitioned cache nodes with near-only cache do not participate in affinity node calculation.
          *
-         * @param cacheName Cache name.
+         * @param cacheId Cache ID.
          * @param topVer Topology version.
          * @return Collection of nodes.
          */
-        Collection<ClusterNode> aliveRemoteCacheNodes(@Nullable String cacheName, final long topVer) {
-            return filter(topVer, aliveRmtCacheNodes.get(maskNull(cacheName)));
+        Collection<ClusterNode> cacheAffinityNodes(int cacheId, final long topVer) {
+            return filter(topVer, affCacheNodes.get(cacheId));
         }
 
         /**
-         * Checks if cache with given name has at least one node with near cache enabled.
+         * Checks if cache with given ID has at least one node with near cache enabled.
          *
-         * @param cacheName Cache name.
+         * @param cacheId Cache ID.
          * @return {@code True} if cache with given name has at least one node with near cache enabled.
          */
-        boolean hasNearCache(@Nullable String cacheName) {
-            return nearEnabledCaches.contains(cacheName);
+        boolean hasNearCache(int cacheId) {
+            return nearEnabledCaches.contains(cacheId);
         }
 
         /**
@@ -2832,51 +2756,10 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             if (leftNode.order() > maxOrder)
                 return;
 
-            filterNodeMap(aliveCacheNodes, leftNode);
-
-            filterNodeMap(aliveRmtCacheNodes, leftNode);
-
             aliveSrvNodesWithCaches.remove(leftNode);
         }
 
         /**
-         * Creates a copy of nodes map without the given node.
-         *
-         * @param map Map to copy.
-         * @param exclNode Node to exclude.
-         */
-        private void filterNodeMap(ConcurrentMap<String, Collection<ClusterNode>> map, final ClusterNode exclNode) {
-            for (String cacheName : registeredCaches.keySet()) {
-                String maskedName = maskNull(cacheName);
-
-                while (true) {
-                    Collection<ClusterNode> oldNodes = map.get(maskedName);
-
-                    if (oldNodes == null || oldNodes.isEmpty())
-                        break;
-
-                    Collection<ClusterNode> newNodes = new ArrayList<>(oldNodes);
-
-                    if (!newNodes.remove(exclNode))
-                        break;
-
-                    if (map.replace(maskedName, oldNodes, newNodes))
-                        break;
-                }
-            }
-        }
-
-        /**
-         * Replaces {@code null} with {@code NULL_CACHE_NAME}.
-         *
-         * @param cacheName Cache name.
-         * @return Masked name.
-         */
-        private String maskNull(@Nullable String cacheName) {
-            return cacheName == null ? NULL_CACHE_NAME : cacheName;
-        }
-
-        /**
          * @param topVer Topology version.
          * @param nodes Nodes.
          * @return Filtered collection (potentially empty, but never {@code null}).

http://git-wip-us.apache.org/repos/asf/ignite/blob/0706f1af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 47abf2f..d0d2df5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -453,7 +453,7 @@ public class GridCacheUtils {
      *      that may have already left).
      */
     public static Collection<ClusterNode> allNodes(GridCacheContext ctx, AffinityTopologyVersion topOrder) {
-        return ctx.discovery().cacheNodes(ctx.namex(), topOrder);
+        return ctx.discovery().cacheNodes(ctx.cacheId(), topOrder);
     }
 
     /**
@@ -486,7 +486,7 @@ public class GridCacheUtils {
      * @return All nodes on which cache with the same name is started.
      */
     public static Collection<ClusterNode> affinityNodes(final GridCacheContext ctx) {
-        return ctx.discovery().cacheAffinityNodes(ctx.namex(), AffinityTopologyVersion.NONE);
+        return ctx.discovery().cacheAffinityNodes(ctx.cacheId(), AffinityTopologyVersion.NONE);
     }
 
     /**
@@ -497,7 +497,7 @@ public class GridCacheUtils {
      * @return Affinity nodes.
      */
     public static Collection<ClusterNode> affinityNodes(GridCacheContext ctx, AffinityTopologyVersion topOrder) {
-        return ctx.discovery().cacheAffinityNodes(ctx.namex(), topOrder);
+        return ctx.discovery().cacheAffinityNodes(ctx.cacheId(), topOrder);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0706f1af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
index 9e85bad..5cba57e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
@@ -196,7 +196,7 @@ public class GridCacheAffinityImpl<K, V> implements Affinity<K> {
         int nodesCnt;
 
         if (!cctx.isLocal())
-            nodesCnt = cctx.discovery().cacheAffinityNodes(cctx.name(), topVer).size();
+            nodesCnt = cctx.discovery().cacheAffinityNodes(cctx.cacheId(), topVer).size();
         else
             nodesCnt = 1;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0706f1af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
index 816132d..3f7fd0d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
@@ -89,7 +89,7 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
     private volatile boolean stopping;
 
     /** A future that will be completed when topology with version topVer will be ready to use. */
-    private GridDhtTopologyFuture topReadyFut;
+    private volatile GridDhtTopologyFuture topReadyFut;
 
     /** */
     private final GridAtomicLong updateSeq = new GridAtomicLong(1);
@@ -217,16 +217,10 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
 
     /** {@inheritDoc} */
     @Override public GridDhtTopologyFuture topologyVersionFuture() {
-        lock.readLock().lock();
-
-        try {
-            assert topReadyFut != null;
+        // TODO
+        assert topReadyFut != null;
 
-            return topReadyFut;
-        }
-        finally {
-            lock.readLock().unlock();
-        }
+        return topReadyFut;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0706f1af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index 519d0fc..fcb6ae5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -1204,8 +1204,8 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
         if (expVer.equals(curVer))
             return false;
 
-        Collection<ClusterNode> cacheNodes0 = ctx.discovery().cacheAffinityNodes(ctx.name(), expVer);
-        Collection<ClusterNode> cacheNodes1 = ctx.discovery().cacheAffinityNodes(ctx.name(), curVer);
+        Collection<ClusterNode> cacheNodes0 = ctx.discovery().cacheAffinityNodes(ctx.cacheId(), expVer);
+        Collection<ClusterNode> cacheNodes1 = ctx.discovery().cacheAffinityNodes(ctx.cacheId(), curVer);
 
         if (!cacheNodes0.equals(cacheNodes1) || ctx.affinity().affinityTopologyVersion().compareTo(curVer) < 0)
             return true;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0706f1af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index 1b4dcc9..b130ed9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -99,7 +99,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     private volatile boolean stopping;
 
     /** A future that will be completed when topology with version topVer will be ready to use. */
-    private GridDhtTopologyFuture topReadyFut;
+    private volatile GridDhtTopologyFuture topReadyFut;
 
     /** */
     private final GridAtomicLong updateSeq = new GridAtomicLong(1);
@@ -311,16 +311,9 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
     /** {@inheritDoc} */
     @Override public GridDhtTopologyFuture topologyVersionFuture() {
-        lock.readLock().lock();
-
-        try {
-            assert topReadyFut != null;
+        assert topReadyFut != null;
 
-            return topReadyFut;
-        }
-        finally {
-            lock.readLock().unlock();
-        }
+        return topReadyFut;
     }
 
     /** {@inheritDoc} */
@@ -752,6 +745,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
             if (part != null)
                 list.add(part);
         }
+
         return list;
     }
 
@@ -831,6 +825,9 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
         List<ClusterNode> affNodes = affAssignment.get(p);
 
+        if (CU.cheatCache(cctx.cacheId()))
+            return affNodes;
+
         lock.readLock().lock();
 
         try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/0706f1af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 0e60ff4..e9c21cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -23,7 +23,6 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -1684,6 +1683,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(ctx.cacheId(), nodeId, req.futureVersion(),
             ctx.deploymentEnabled());
 
+        res.partition(req.partition());
+
         assert !req.returnValue() || (req.operation() == TRANSFORM || req.size() == 1);
 
         GridDhtAtomicAbstractUpdateFuture dhtFut = null;
@@ -1704,7 +1705,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             try {
                 GridDhtPartitionTopology top = topology();
 
-                top.readLock();
+                if (!CU.cheatCache(ctx.cacheId()))
+                    top.readLock();
 
                 try {
                     if (top.stopping()) {
@@ -1737,7 +1739,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
                         if (ver == null) {
                             // Assign next version for update inside entries lock.
-                            ver = ctx.versions().next(top.topologyVersion());
+                            ver = ctx.versions().next(top.topologyVersion(), req.partition());
 
                             if (hasNear)
                                 res.nearVersion(ver);
@@ -1819,7 +1821,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                         remap = true;
                 }
                 finally {
-                    top.readUnlock();
+                    if (!CU.cheatCache(ctx.cacheId()))
+                        top.readUnlock();
                 }
             }
             catch (GridCacheEntryRemovedException e) {
@@ -2344,7 +2347,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
         AffinityTopologyVersion topVer = req.topologyVersion();
 
-        boolean checkReaders = hasNear || ctx.discovery().hasNearCache(name(), topVer);
+        boolean checkReaders = hasNear || ctx.discovery().hasNearCache(ctx.cacheId(), topVer);
 
         boolean readersOnly = false;
 
@@ -2579,7 +2582,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
         AffinityTopologyVersion topVer = req.topologyVersion();
 
-        boolean checkReaders = hasNear || ctx.discovery().hasNearCache(name(), topVer);
+        boolean checkReaders = hasNear || ctx.discovery().hasNearCache(ctx.cacheId(), topVer);
 
         CacheStorePartialUpdateException storeErr = null;
 
@@ -2818,6 +2821,9 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                 try {
                     GridDhtCacheEntry entry = entryExx(key, topVer);
 
+                    if (CU.cheatCache(ctx.cacheId())) // TODO
+                        return Collections.singletonList(entry);
+
                     GridUnsafe.monitorEnter(entry);
 
                     if (entry.obsolete())
@@ -2893,6 +2899,9 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param topVer Topology version.
      */
     private void unlockEntries(Collection<GridDhtCacheEntry> locked, AffinityTopologyVersion topVer) {
+        if (CU.cheatCache(ctx.cacheId()))
+            return;
+
         // Process deleted entries before locks release.
         assert ctx.deferredDelete() : this;
 
@@ -2904,7 +2913,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             for (GridCacheMapEntry entry : locked) {
                 if (entry != null && entry.deleted()) {
                     if (skip == null)
-                        skip = new HashSet<>(locked.size(), 1.0f);
+                        skip = U.newHashSet(locked.size());
 
                     skip.add(entry.key());
                 }
@@ -3050,7 +3059,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
             AffinityTopologyVersion topVer = updateReq.topologyVersion();
 
-            Collection<ClusterNode> nodes = ctx.kernalContext().discovery().cacheAffinityNodes(name(), topVer);
+            Collection<ClusterNode> nodes = ctx.kernalContext().discovery().cacheAffinityNodes(ctx.cacheId(), topVer);
 
             // We are on primary node for some key.
             assert !nodes.isEmpty() : "Failed to find affinity nodes [name=" + name() + ", topVer=" + topVer +

http://git-wip-us.apache.org/repos/asf/ignite/blob/0706f1af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
index bd231cf..f148cbb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
@@ -386,54 +386,58 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
 
     /** {@inheritDoc} */
     @Override protected void mapOnTopology() {
-        cache.topology().readLock();
-
         AffinityTopologyVersion topVer = null;
 
-        try {
-            if (cache.topology().stopping()) {
-                onDone(new IgniteCheckedException("Failed to perform cache operation (cache is stopped): " +
-                    cache.name()));
+        if (!CU.cheatCache(cctx.cacheId())) {
+            cache.topology().readLock();
 
-                return;
-            }
+            try {
+                if (cache.topology().stopping()) {
+                    onDone(new IgniteCheckedException("Failed to perform cache operation (cache is stopped): " +
+                        cache.name()));
 
-            GridDhtTopologyFuture fut = cache.topology().topologyVersionFuture();
+                    return;
+                }
 
-            if (fut.isDone()) {
-                Throwable err = fut.validateCache(cctx);
+                GridDhtTopologyFuture fut = cache.topology().topologyVersionFuture();
 
-                if (err != null) {
-                    onDone(err);
+                if (fut.isDone()) {
+                    Throwable err = fut.validateCache(cctx);
 
-                    return;
-                }
+                    if (err != null) {
+                        onDone(err);
 
-                topVer = fut.topologyVersion();
-            }
-            else {
-                if (waitTopFut) {
-                    assert !topLocked : this;
-
-                    fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
-                        @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
-                            cctx.kernalContext().closure().runLocalSafe(new Runnable() {
-                                @Override public void run() {
-                                    mapOnTopology();
-                                }
-                            });
-                        }
-                    });
+                        return;
+                    }
+
+                    topVer = fut.topologyVersion();
                 }
-                else
-                    onDone(new GridCacheTryPutFailedException());
+                else {
+                    if (waitTopFut) {
+                        assert !topLocked : this;
+
+                        fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
+                            @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
+                                cctx.kernalContext().closure().runLocalSafe(new Runnable() {
+                                    @Override public void run() {
+                                        mapOnTopology();
+                                    }
+                                });
+                            }
+                        });
+                    }
+                    else
+                        onDone(new GridCacheTryPutFailedException());
 
-                return;
+                    return;
+                }
+            }
+            finally {
+                cache.topology().readUnlock();
             }
         }
-        finally {
-            cache.topology().readUnlock();
-        }
+        else
+            topVer = cache.topology().topologyVersionFuture().topologyVersion();
 
         map(topVer);
     }
@@ -461,7 +465,7 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
             updVer = this.updVer;
 
             if (updVer == null) {
-                updVer = cctx.versions().next(topVer);
+                updVer = futVer;
 
                 if (log.isDebugEnabled())
                     log.debug("Assigned fast-map version for update on near node: " + updVer);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0706f1af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
index cd64117..0b917e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
@@ -495,54 +495,58 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
 
     /** {@inheritDoc} */
     @Override protected void mapOnTopology() {
-        cache.topology().readLock();
-
         AffinityTopologyVersion topVer = null;
 
-        try {
-            if (cache.topology().stopping()) {
-                onDone(new IgniteCheckedException("Failed to perform cache operation (cache is stopped): " +
-                    cache.name()));
+        if (!CU.cheatCache(cctx.cacheId())) {
+            cache.topology().readLock();
 
-                return;
-            }
+            try {
+                if (cache.topology().stopping()) {
+                    onDone(new IgniteCheckedException("Failed to perform cache operation (cache is stopped): " +
+                        cache.name()));
 
-            GridDhtTopologyFuture fut = cache.topology().topologyVersionFuture();
+                    return;
+                }
 
-            if (fut.isDone()) {
-                Throwable err = fut.validateCache(cctx);
+                GridDhtTopologyFuture fut = cache.topology().topologyVersionFuture();
 
-                if (err != null) {
-                    onDone(err);
+                if (fut.isDone()) {
+                    Throwable err = fut.validateCache(cctx);
 
-                    return;
-                }
+                    if (err != null) {
+                        onDone(err);
 
-                topVer = fut.topologyVersion();
-            }
-            else {
-                if (waitTopFut) {
-                    assert !topLocked : this;
-
-                    fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
-                        @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
-                            cctx.kernalContext().closure().runLocalSafe(new Runnable() {
-                                @Override public void run() {
-                                    mapOnTopology();
-                                }
-                            });
-                        }
-                    });
+                        return;
+                    }
+
+                    topVer = fut.topologyVersion();
                 }
-                else
-                    onDone(new GridCacheTryPutFailedException());
+                else {
+                    if (waitTopFut) {
+                        assert !topLocked : this;
+
+                        fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
+                            @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
+                                cctx.kernalContext().closure().runLocalSafe(new Runnable() {
+                                    @Override public void run() {
+                                        mapOnTopology();
+                                    }
+                                });
+                            }
+                        });
+                    }
+                    else
+                        onDone(new GridCacheTryPutFailedException());
 
-                return;
+                    return;
+                }
+            }
+            finally {
+                cache.topology().readUnlock();
             }
         }
-        finally {
-            cache.topology().readUnlock();
-        }
+        else
+            topVer = cache.topology().topologyVersionFuture().topologyVersion();
 
         map(topVer, null);
     }
@@ -635,7 +639,8 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
             updVer = this.updVer;
 
             if (updVer == null) {
-                updVer = cctx.versions().next(topVer);
+                //updVer = cctx.versions().next(topVer);
+                updVer = futVer;
 
                 if (log.isDebugEnabled())
                     log.debug("Assigned fast-map version for update on near node: " + updVer);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0706f1af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
index 2e38733..22e01ae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
@@ -105,6 +105,9 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
     /** Near expire times. */
     private GridLongList nearExpireTimes;
 
+    /** Partition ID. */
+    private int partId = -1;
+
     /**
      * Empty constructor required by {@link Externalizable}.
      */
@@ -154,6 +157,13 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
     }
 
     /**
+     * @param partId Partition ID for proper striping on near node.
+     */
+    public void partition(int partId) {
+        this.partId = partId;
+    }
+
+    /**
      * Sets update error.
      *
      * @param err Error.
@@ -431,6 +441,11 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
     }
 
     /** {@inheritDoc} */
+    @Override public int partition() {
+        return partId;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean addDeploymentInfo() {
         return addDepInfo;
     }
@@ -510,12 +525,18 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
                 writer.incrementState();
 
             case 12:
-                if (!writer.writeCollection("remapKeys", remapKeys, MessageCollectionItemType.MSG))
+                if (!writer.writeInt("partId", partId))
                     return false;
 
                 writer.incrementState();
 
             case 13:
+                if (!writer.writeCollection("remapKeys", remapKeys, MessageCollectionItemType.MSG))
+                    return false;
+
+                writer.incrementState();
+
+            case 14:
                 if (!writer.writeMessage("ret", ret))
                     return false;
 
@@ -610,7 +631,7 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
                 reader.incrementState();
 
             case 12:
-                remapKeys = reader.readCollection("remapKeys", MessageCollectionItemType.MSG);
+                partId = reader.readInt("partId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -618,6 +639,14 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
                 reader.incrementState();
 
             case 13:
+                remapKeys = reader.readCollection("remapKeys", MessageCollectionItemType.MSG);
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 14:
                 ret = reader.readMessage("ret");
 
                 if (!reader.isLastRead())
@@ -637,7 +666,7 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 14;
+        return 15;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0706f1af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java
index 95aab74..ccc17e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java
@@ -354,4 +354,4 @@ public class GridCacheVersion implements Message, Comparable<GridCacheVersion>,
             ", order=" + order() +
             ", nodeOrder=" + nodeOrder() + ']';
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0706f1af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
index 9be8b50..fea5ee9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.version;
 
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicLongArray;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.DiscoveryEvent;
@@ -64,6 +65,9 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
     private GridCacheVersion ISOLATED_STREAMER_VER;
 
     /** */
+    private final AtomicLongArray orders = new AtomicLongArray(Runtime.getRuntime().availableProcessors());
+
+    /** */
     private final GridLocalEventListener discoLsnr = new GridLocalEventListener() {
         @Override public void onEvent(Event evt) {
             assert evt.type() == EVT_NODE_METRICS_UPDATED;
@@ -195,7 +199,8 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
      * @return Next version based on current topology.
      */
     public GridCacheVersion next() {
-        return next(cctx.kernalContext().discovery().topologyVersion(), true, false, dataCenterId);
+        return next(cctx.kernalContext().discovery().topologyVersion(), true, false, dataCenterId,
+            -1);
     }
 
     /**
@@ -203,7 +208,8 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
      * @return Next version based on current topology with given data center id.
      */
     public GridCacheVersion next(byte dataCenterId) {
-        return next(cctx.kernalContext().discovery().topologyVersion(), true, false, dataCenterId);
+        return next(cctx.kernalContext().discovery().topologyVersion(), true, false, dataCenterId,
+            -1);
     }
 
     /**
@@ -215,7 +221,21 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
      * @return Next version based on given topology version.
      */
     public GridCacheVersion next(AffinityTopologyVersion topVer) {
-        return next(topVer.topologyVersion(), true, false, dataCenterId);
+        return next(topVer.topologyVersion(), true, false, dataCenterId,
+            -1);
+    }
+
+    /**
+     * Gets next version based on given topology version. Given value should be
+     * real topology version calculated as number of grid topology changes and
+     * obtained from discovery manager.
+     *
+     * @param topVer Topology version for which new version should be obtained.
+     * @param part Partition.
+     * @return Next version based on given topology version.
+     */
+    public GridCacheVersion next(AffinityTopologyVersion topVer, int part) {
+        return next(topVer.topologyVersion(), true, false, dataCenterId, part);
     }
 
     /**
@@ -224,7 +244,8 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
      * @return Next version for cache store operations.
      */
     public GridCacheVersion nextForLoad() {
-        return next(cctx.kernalContext().discovery().topologyVersion(), true, true, dataCenterId);
+        return next(cctx.kernalContext().discovery().topologyVersion(), true, true, dataCenterId,
+            -1);
     }
 
     /**
@@ -233,7 +254,8 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
      * @return Next version for cache store operations.
      */
     public GridCacheVersion nextForLoad(AffinityTopologyVersion topVer) {
-        return next(topVer.topologyVersion(), true, true, dataCenterId);
+        return next(topVer.topologyVersion(), true, true, dataCenterId,
+            -1);
     }
 
     /**
@@ -242,7 +264,8 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
      * @return Next version for cache store operations.
      */
     public GridCacheVersion nextForLoad(GridCacheVersion ver) {
-        return next(ver.topologyVersion(), false, true, dataCenterId);
+        return next(ver.topologyVersion(), false, true, dataCenterId,
+            -1);
     }
 
     /**
@@ -252,7 +275,8 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
      * @return Next version based on given cache version.
      */
     public GridCacheVersion next(GridCacheVersion ver) {
-        return next(ver.topologyVersion(), false, false, dataCenterId);
+        return next(ver.topologyVersion(), false, false, dataCenterId,
+            -1);
     }
 
     /**
@@ -265,9 +289,16 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
      * @param addTime If {@code true} then adds to the given topology version number of seconds
      *        from the start time of the first grid node.
      * @param dataCenterId Data center id.
+     * @param part
      * @return New lock order.
      */
-    private GridCacheVersion next(long topVer, boolean addTime, boolean forLoad, byte dataCenterId) {
+    private GridCacheVersion next(
+        long topVer,
+        boolean addTime,
+        boolean forLoad,
+        byte dataCenterId,
+        int part
+    ) {
         if (topVer == -1)
             topVer = cctx.kernalContext().discovery().topologyVersion();
 
@@ -282,7 +313,8 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
 
         int locNodeOrder = (int)cctx.localNode().order();
 
-        long ord = forLoad ? loadOrder.incrementAndGet() : order.incrementAndGet();
+        long ord = forLoad ? loadOrder.incrementAndGet() :
+            part == -1 ? order.incrementAndGet() : (orders.incrementAndGet(part % orders.length()) | (((long)part) << 48 ));
 
         GridCacheVersion next = new GridCacheVersion(
             (int)topVer,
@@ -291,7 +323,7 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
             locNodeOrder,
             dataCenterId);
 
-        last = next;
+        // last = next; // TODO
 
         return next;
     }
@@ -304,4 +336,4 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
     public GridCacheVersion last() {
         return last;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0706f1af/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
index 0764316..36178f3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
@@ -273,31 +273,33 @@ public class GridClockSyncProcessor extends GridProcessorAdapter {
      * @return Adjusted time.
      */
     public long adjustedTime(long topVer) {
-        T2<GridClockDeltaVersion, GridClockDeltaSnapshot> fastSnap = lastSnapshot;
-
-        GridClockDeltaSnapshot snap;
-
-        if (fastSnap != null && fastSnap.get1().topologyVersion() == topVer)
-            snap = fastSnap.get2();
-        else {
-            // Get last synchronized time on given topology version.
-            Map.Entry<GridClockDeltaVersion, GridClockDeltaSnapshot> entry = timeSyncHistory().lowerEntry(
-                new GridClockDeltaVersion(0, topVer + 1));
-
-            snap = entry == null ? null : entry.getValue();
-        }
-
-        long now = clockSrc.currentTimeMillis();
-
-        if (snap == null)
-            return now;
-
-        Long delta = snap.deltas().get(ctx.localNodeId());
-
-        if (delta == null)
-            delta = 0L;
-
-        return now + delta;
+//        T2<GridClockDeltaVersion, GridClockDeltaSnapshot> fastSnap = lastSnapshot;
+//
+//        GridClockDeltaSnapshot snap;
+//
+//        if (fastSnap != null && fastSnap.get1().topologyVersion() == topVer)
+//            snap = fastSnap.get2();
+//        else {
+//            // Get last synchronized time on given topology version.
+//            Map.Entry<GridClockDeltaVersion, GridClockDeltaSnapshot> entry = timeSyncHistory().lowerEntry(
+//                new GridClockDeltaVersion(0, topVer + 1));
+//
+//            snap = entry == null ? null : entry.getValue();
+//        }
+//
+//        long now = clockSrc.currentTimeMillis();
+//
+//        if (snap == null)
+//            return now;
+//
+//        Long delta = snap.deltas().get(ctx.localNodeId());
+//
+//        if (delta == null)
+//            delta = 0L;
+//
+//        return now + delta;
+
+        return System.currentTimeMillis();
     }
 
     /**
@@ -478,4 +480,4 @@ public class GridClockSyncProcessor extends GridProcessorAdapter {
                 curr.onNodeLeft(nodeId);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0706f1af/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
index 31b4bc7..03cbdbd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
@@ -23,16 +23,11 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import org.apache.ignite.Ignite;
-import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.events.EventType;
-import org.apache.ignite.internal.IgniteKernal;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
-import org.apache.ignite.internal.processors.cache.GridCacheUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.lang.IgnitePredicate;
@@ -158,8 +153,6 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
             stopTempNodes();
 
             latch.await();
-
-            validateAlives();
         }
     }
 
@@ -200,55 +193,6 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
     }
 
     /**
-     * Validates that all node collections contain actual information.
-     */
-    @SuppressWarnings("SuspiciousMethodCalls")
-    private void validateAlives() {
-        for (Ignite g : alive) {
-            log.info("Validate node: " + g.name());
-
-            assertEquals("Unexpected nodes number for node: " + g.name(), PERM_NODES_CNT, g.cluster().nodes().size());
-        }
-
-        for (final Ignite g : alive) {
-            IgniteKernal k = (IgniteKernal)g;
-
-            GridDiscoveryManager discoMgr = k.context().discovery();
-
-            final Collection<ClusterNode> currTop = g.cluster().nodes();
-
-            long currVer = discoMgr.topologyVersion();
-
-            long startVer = discoMgr.localNode().order();
-
-            for (long v = currVer; v > currVer - GridDiscoveryManager.DISCOVERY_HISTORY_SIZE && v >= startVer; v--) {
-                F.forAll(discoMgr.aliveCacheNodes(null, new AffinityTopologyVersion(v)),
-                    new IgnitePredicate<ClusterNode>() {
-                        @Override public boolean apply(ClusterNode e) {
-                            return currTop.contains(e);
-                        }
-                    });
-
-                F.forAll(discoMgr.aliveRemoteCacheNodes(null, new AffinityTopologyVersion(v)),
-                    new IgnitePredicate<ClusterNode>() {
-                        @Override public boolean apply(ClusterNode e) {
-                            return currTop.contains(e) || g.cluster().localNode().equals(e);
-                        }
-                    });
-
-                GridCacheSharedContext<?, ?> ctx = k.context().cache().context();
-
-                ClusterNode oldest =
-                    ctx.discovery().oldestAliveCacheServerNode(new AffinityTopologyVersion(currVer));
-
-                assertNotNull(oldest);
-
-                assertTrue(currTop.contains(oldest));
-            }
-        }
-    }
-
-    /**
      * Starts temporary nodes.
      *
      * @throws Exception If failed.
@@ -293,4 +237,4 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
             G.stop(g.name(), false);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0706f1af/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
index c9179d4..230a3bc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
@@ -211,4 +211,4 @@ public abstract class GridDiscoveryManagerSelfTest extends GridCommonAbstractTes
     public static class ClientDiscovery extends GridDiscoveryManagerSelfTest {
         // No-op.
     }
-}
\ No newline at end of file
+}


[41/50] [abbrv] ignite git commit: IGNITE-4367 .NET: Fix flaky tests - fix race in ReconnectTest.TestClusterRestart

Posted by yz...@apache.org.
IGNITE-4367 .NET: Fix flaky tests - fix race in ReconnectTest.TestClusterRestart


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

Branch: refs/heads/ignite-comm-balance-master
Commit: df725e837da06bc319f9655d138909d1fd96ffbc
Parents: c040c37
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Dec 26 17:35:51 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Dec 26 17:35:51 2016 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs          | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/df725e83/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 fdf64a3..5e9f788 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs
@@ -72,11 +72,14 @@ namespace Apache.Ignite.Core.Tests
 
             // Start the server and wait for reconnect.
             Ignition.Start(serverCfg);
+
+            // Wait a bit for notifications.
+            Thread.Sleep(100);
+
+            // Check reconnect task.
             Assert.IsTrue(ex.ClientReconnectTask.Result);
 
             // Check the event args.
-            Thread.Sleep(100);  // Wait for event handler
-
             Assert.IsNotNull(eventArgs);
             Assert.IsTrue(eventArgs.HasClusterRestarted);
 


[28/50] [abbrv] ignite git commit: Web console beta-7.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/generator-java.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/generator-java.js b/modules/web-console/frontend/app/modules/configuration/generator/generator-java.js
deleted file mode 100644
index 296b942..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/generator-java.js
+++ /dev/null
@@ -1,3617 +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.
- */
-
-// Java generation entry point.
-const $generatorJava = {};
-
-/**
- * Translate some value to valid java code.
- *
- * @param val Value to convert.
- * @param type Value type.
- * @returns {*} String with value that will be valid for java.
- */
-$generatorJava.toJavaCode = function(val, type) {
-    if (val === null)
-        return 'null';
-
-    if (type === 'raw')
-        return val;
-
-    if (type === 'class')
-        return val + '.class';
-
-    if (type === 'float')
-        return val + 'f';
-
-    if (type === 'path')
-        return '"' + val.replace(/\\/g, '\\\\') + '"';
-
-    if (type)
-        return type + '.' + val;
-
-    if (typeof (val) === 'string')
-        return '"' + val.replace('"', '\\"') + '"';
-
-    if (typeof (val) === 'number' || typeof (val) === 'boolean')
-        return String(val);
-
-    return 'Unknown type: ' + typeof (val) + ' (' + val + ')';
-};
-
-/**
- * @param propName Property name.
- * @param setterName Optional concrete setter name.
- * @returns Property setter with name by java conventions.
- */
-$generatorJava.setterName = function(propName, setterName) {
-    return setterName ? setterName : $generatorCommon.toJavaName('set', propName);
-};
-
-// Add constructor argument
-$generatorJava.constructorArg = function(obj, propName, dflt, notFirst, opt) {
-    const v = (obj ? obj[propName] : null) || dflt;
-
-    if ($generatorCommon.isDefinedAndNotEmpty(v))
-        return (notFirst ? ', ' : '') + $generatorJava.toJavaCode(v);
-    else if (!opt)
-        return notFirst ? ', null' : 'null';
-
-    return '';
-};
-
-/**
- * Add variable declaration.
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param varFullType Variable full class name to be added to imports.
- * @param varFullActualType Variable actual full class name to be added to imports.
- * @param varFullGenericType1 Optional full class name of first generic.
- * @param varFullGenericType2 Optional full class name of second generic.
- * @param subClass If 'true' then variable will be declared as anonymous subclass.
- */
-$generatorJava.declareVariable = function(res, varName, varFullType, varFullActualType, varFullGenericType1, varFullGenericType2, subClass) {
-    res.emptyLineIfNeeded();
-
-    const varType = res.importClass(varFullType);
-
-    const varNew = !res.vars[varName];
-
-    if (varNew)
-        res.vars[varName] = true;
-
-    if (varFullActualType && varFullGenericType1) {
-        const varActualType = res.importClass(varFullActualType);
-        const varGenericType1 = res.importClass(varFullGenericType1);
-        let varGenericType2 = null;
-
-        if (varFullGenericType2)
-            varGenericType2 = res.importClass(varFullGenericType2);
-
-        res.line((varNew ? (varType + '<' + varGenericType1 + (varGenericType2 ? ', ' + varGenericType2 : '') + '> ') : '') +
-            varName + ' = new ' + varActualType + '<>();');
-    }
-    else
-        res.line((varNew ? (varType + ' ') : '') + varName + ' = new ' + varType + '()' + (subClass ? ' {' : ';'));
-
-    if (!subClass)
-        res.needEmptyLine = true;
-
-    return varName;
-};
-
-/**
- * Add local variable declaration.
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param varFullType Variable full class name to be added to imports.
- */
-$generatorJava.declareVariableLocal = function(res, varName, varFullType) {
-    const varType = res.importClass(varFullType);
-
-    res.line(varType + ' ' + varName + ' = new ' + varType + '();');
-
-    res.needEmptyLine = true;
-};
-
-/**
- * Add custom variable declaration.
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param varFullType Variable full class name to be added to imports.
- * @param varExpr Custom variable creation expression.
- * @param modifier Additional variable modifier.
- */
-$generatorJava.declareVariableCustom = function(res, varName, varFullType, varExpr, modifier) {
-    const varType = res.importClass(varFullType);
-
-    const varNew = !res.vars[varName];
-
-    if (varNew)
-        res.vars[varName] = true;
-
-    res.line((varNew ? ((modifier ? modifier + ' ' : '') + varType + ' ') : '') + varName + ' = ' + varExpr + ';');
-
-    res.needEmptyLine = true;
-};
-
-/**
- * Add array variable declaration.
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param varFullType Variable full class name to be added to imports.
- * @param length Array length.
- */
-$generatorJava.declareVariableArray = function(res, varName, varFullType, length) {
-    const varType = res.importClass(varFullType);
-
-    const varNew = !res.vars[varName];
-
-    if (varNew)
-        res.vars[varName] = true;
-
-    res.line((varNew ? (varType + '[] ') : '') + varName + ' = new ' + varType + '[' + length + '];');
-
-    res.needEmptyLine = true;
-};
-
-/**
- * Clear list of declared variables.
- *
- * @param res
- */
-$generatorJava.resetVariables = function(res) {
-    res.vars = {};
-};
-
-/**
- * Add property via setter / property name.
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param obj Source object with data.
- * @param propName Property name to take from source object.
- * @param dataType Optional info about property data type.
- * @param setterName Optional special setter name.
- * @param dflt Optional default value.
- */
-$generatorJava.property = function(res, varName, obj, propName, dataType, setterName, dflt) {
-    if (!_.isNil(obj)) {
-        const val = obj[propName];
-
-        if ($generatorCommon.isDefinedAndNotEmpty(val)) {
-            const missDflt = _.isNil(dflt);
-
-            // Add to result if no default provided or value not equals to default.
-            if (missDflt || (!missDflt && val !== dflt)) {
-                res.line(varName + '.' + $generatorJava.setterName(propName, setterName) +
-                    '(' + $generatorJava.toJavaCode(val, dataType) + ');');
-
-                return true;
-            }
-        }
-    }
-
-    return false;
-};
-
-/**
- * Add enum property via setter / property name.
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param obj Source object with data.
- * @param propName Property name to take from source object.
- * @param dataType Name of enum class
- * @param setterName Optional special setter name.
- * @param dflt Optional default value.
- */
-$generatorJava.enumProperty = function(res, varName, obj, propName, dataType, setterName, dflt) {
-    const val = obj[propName];
-
-    if ($generatorCommon.isDefinedAndNotEmpty(val)) {
-        const missDflt = _.isNil(dflt);
-
-        // Add to result if no default provided or value not equals to default.
-        if (missDflt || (!missDflt && val !== dflt)) {
-            res.line(varName + '.' + $generatorJava.setterName(propName, setterName) +
-                '(' + $generatorJava.toJavaCode(val, dataType ? res.importClass(dataType) : null) + ');');
-
-            return true;
-        }
-    }
-
-    return false;
-};
-
-// Add property for class name.
-$generatorJava.classNameProperty = function(res, varName, obj, propName) {
-    const val = obj[propName];
-
-    if (!_.isNil(val)) {
-        res.line(varName + '.' + $generatorJava.setterName(propName) +
-            '("' + $generatorCommon.JavaTypes.fullClassName(val) + '");');
-    }
-};
-
-/**
- * Add list property.
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param obj Source object with data.
- * @param propName Property name to take from source object.
- * @param dataType Optional data type.
- * @param setterName Optional setter name.
- */
-$generatorJava.listProperty = function(res, varName, obj, propName, dataType, setterName) {
-    const val = obj[propName];
-
-    if (val && val.length > 0) {
-        res.emptyLineIfNeeded();
-
-        res.importClass('java.util.Arrays');
-
-        $generatorJava.fxVarArgs(res, varName + '.' + $generatorJava.setterName(propName, setterName), false,
-            _.map(val, (v) => $generatorJava.toJavaCode(v, dataType)), '(Arrays.asList(', '))');
-
-        res.needEmptyLine = true;
-    }
-};
-
-/**
- * Add function with varargs arguments.
- *
- * @param res Resulting output with generated code.
- * @param fx Function name.
- * @param quote Whether to quote arguments.
- * @param args Array with arguments.
- * @param startBlock Optional start block string.
- * @param endBlock Optional end block string.
- * @param startQuote Start quote string.
- * @param endQuote End quote string.
- */
-$generatorJava.fxVarArgs = function(res, fx, quote, args, startBlock = '(', endBlock = ')', startQuote = '"', endQuote = '"') {
-    const quoteArg = (arg) => quote ? startQuote + arg + endQuote : arg;
-
-    if (args.length === 1)
-        res.append(fx + startBlock + quoteArg(args[0]) + endBlock + ';');
-    else {
-        res.startBlock(fx + startBlock);
-
-        const len = args.length - 1;
-
-        _.forEach(args, (arg, ix) => res.line(quoteArg(arg) + (ix < len ? ', ' : '')));
-
-        res.endBlock(endBlock + ';');
-    }
-};
-
-/**
- * Add array property.
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param obj Source object with data.
- * @param propName Property name to take from source object.
- * @param setterName Optional setter name.
- */
-$generatorJava.arrayProperty = function(res, varName, obj, propName, setterName) {
-    const val = obj[propName];
-
-    if (val && val.length > 0) {
-        res.emptyLineIfNeeded();
-
-        $generatorJava.fxVarArgs(res, varName + '.' + $generatorJava.setterName(propName, setterName), false,
-            _.map(val, (v) => 'new ' + res.importClass(v) + '()'), '({ ', ' });');
-
-        res.needEmptyLine = true;
-    }
-};
-
-/**
- * Add multi-param property (setter with several arguments).
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param obj Source object with data.
- * @param propName Property name to take from source object.
- * @param dataType Optional data type.
- * @param setterName Optional setter name.
- */
-$generatorJava.multiparamProperty = function(res, varName, obj, propName, dataType, setterName) {
-    const val = obj[propName];
-
-    if (val && val.length > 0) {
-        $generatorJava.fxVarArgs(res, varName + '.' + $generatorJava.setterName(propName, setterName), false,
-            _.map(val, (v) => $generatorJava.toJavaCode(dataType === 'class' ? res.importClass(v) : v, dataType)));
-    }
-};
-
-/**
- * Add complex bean.
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param bean
- * @param beanPropName Bean property name.
- * @param beanVarName
- * @param beanClass Bean class.
- * @param props
- * @param createBeanAlthoughNoProps If 'true' then create empty bean.
- */
-$generatorJava.beanProperty = function(res, varName, bean, beanPropName, beanVarName, beanClass, props, createBeanAlthoughNoProps) {
-    if (bean && $generatorCommon.hasProperty(bean, props)) {
-        res.emptyLineIfNeeded();
-
-        $generatorJava.declareVariable(res, beanVarName, beanClass);
-
-        _.forIn(props, function(descr, propName) {
-            if (props.hasOwnProperty(propName)) {
-                if (descr) {
-                    switch (descr.type) {
-                        case 'list':
-                            $generatorJava.listProperty(res, beanVarName, bean, propName, descr.elementsType, descr.setterName);
-                            break;
-
-                        case 'array':
-                            $generatorJava.arrayProperty(res, beanVarName, bean, propName, descr.setterName);
-                            break;
-
-                        case 'enum':
-                            $generatorJava.enumProperty(res, beanVarName, bean, propName, descr.enumClass, descr.setterName, descr.dflt);
-                            break;
-
-                        case 'float':
-                            $generatorJava.property(res, beanVarName, bean, propName, 'float', descr.setterName);
-                            break;
-
-                        case 'path':
-                            $generatorJava.property(res, beanVarName, bean, propName, 'path', descr.setterName);
-                            break;
-
-                        case 'raw':
-                            $generatorJava.property(res, beanVarName, bean, propName, 'raw', descr.setterName);
-                            break;
-
-                        case 'propertiesAsList':
-                            const val = bean[propName];
-
-                            if (val && val.length > 0) {
-                                $generatorJava.declareVariable(res, descr.propVarName, 'java.util.Properties');
-
-                                _.forEach(val, function(nameAndValue) {
-                                    const eqIndex = nameAndValue.indexOf('=');
-
-                                    if (eqIndex >= 0) {
-                                        res.line(descr.propVarName + '.setProperty(' +
-                                            '"' + nameAndValue.substring(0, eqIndex) + '", ' +
-                                            '"' + nameAndValue.substr(eqIndex + 1) + '");');
-                                    }
-                                });
-
-                                res.needEmptyLine = true;
-
-                                res.line(beanVarName + '.' + $generatorJava.setterName(propName) + '(' + descr.propVarName + ');');
-                            }
-                            break;
-
-                        case 'bean':
-                            if ($generatorCommon.isDefinedAndNotEmpty(bean[propName]))
-                                res.line(beanVarName + '.' + $generatorJava.setterName(propName) + '(new ' + res.importClass(bean[propName]) + '());');
-
-                            break;
-
-                        default:
-                            $generatorJava.property(res, beanVarName, bean, propName, null, descr.setterName, descr.dflt);
-                    }
-                }
-                else
-                    $generatorJava.property(res, beanVarName, bean, propName);
-            }
-        });
-
-        res.needEmptyLine = true;
-
-        res.line(varName + '.' + $generatorJava.setterName(beanPropName) + '(' + beanVarName + ');');
-
-        res.needEmptyLine = true;
-    }
-    else if (createBeanAlthoughNoProps) {
-        res.emptyLineIfNeeded();
-        res.line(varName + '.' + $generatorJava.setterName(beanPropName) + '(new ' + res.importClass(beanClass) + '());');
-
-        res.needEmptyLine = true;
-    }
-};
-
-/**
- * Add eviction policy.
- *
- * @param res Resulting output with generated code.
- * @param varName Current using variable name.
- * @param evtPlc Data to add.
- * @param propName Name in source data.
- */
-$generatorJava.evictionPolicy = function(res, varName, evtPlc, propName) {
-    if (evtPlc && evtPlc.kind) {
-        const evictionPolicyDesc = $generatorCommon.EVICTION_POLICIES[evtPlc.kind];
-
-        const obj = evtPlc[evtPlc.kind.toUpperCase()];
-
-        $generatorJava.beanProperty(res, varName, obj, propName, propName,
-            evictionPolicyDesc.className, evictionPolicyDesc.fields, true);
-    }
-};
-
-// Generate cluster general group.
-$generatorJava.clusterGeneral = function(cluster, clientNearCfg, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorJava.declareVariable(res, 'cfg', 'org.apache.ignite.configuration.IgniteConfiguration');
-
-    $generatorJava.property(res, 'cfg', cluster, 'name', null, 'setGridName');
-    res.needEmptyLine = true;
-
-    $generatorJava.property(res, 'cfg', cluster, 'localHost');
-    res.needEmptyLine = true;
-
-    if (clientNearCfg) {
-        res.line('cfg.setClientMode(true);');
-
-        res.needEmptyLine = true;
-    }
-
-    if (cluster.discovery) {
-        const d = cluster.discovery;
-
-        $generatorJava.declareVariable(res, 'discovery', 'org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi');
-
-        switch (d.kind) {
-            case 'Multicast':
-                $generatorJava.beanProperty(res, 'discovery', d.Multicast, 'ipFinder', 'ipFinder',
-                    'org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder',
-                    {
-                        multicastGroup: null,
-                        multicastPort: null,
-                        responseWaitTime: null,
-                        addressRequestAttempts: null,
-                        localAddress: null,
-                        addresses: {type: 'list'}
-                    }, true);
-
-                break;
-
-            case 'Vm':
-                $generatorJava.beanProperty(res, 'discovery', d.Vm, 'ipFinder', 'ipFinder',
-                    'org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder',
-                    {addresses: {type: 'list'}}, true);
-
-                break;
-
-            case 'S3':
-                $generatorJava.beanProperty(res, 'discovery', d.S3, 'ipFinder', 'ipFinder',
-                    'org.apache.ignite.spi.discovery.tcp.ipfinder.s3.TcpDiscoveryS3IpFinder', {bucketName: null}, true);
-
-                break;
-
-            case 'Cloud':
-                $generatorJava.beanProperty(res, 'discovery', d.Cloud, 'ipFinder', 'ipFinder',
-                    'org.apache.ignite.spi.discovery.tcp.ipfinder.cloud.TcpDiscoveryCloudIpFinder',
-                    {
-                        credential: null,
-                        credentialPath: null,
-                        identity: null,
-                        provider: null,
-                        regions: {type: 'list'},
-                        zones: {type: 'list'}
-                    }, true);
-
-                break;
-
-            case 'GoogleStorage':
-                $generatorJava.beanProperty(res, 'discovery', d.GoogleStorage, 'ipFinder', 'ipFinder',
-                    'org.apache.ignite.spi.discovery.tcp.ipfinder.gce.TcpDiscoveryGoogleStorageIpFinder',
-                    {
-                        projectName: null,
-                        bucketName: null,
-                        serviceAccountP12FilePath: null,
-                        serviceAccountId: null
-                    }, true);
-
-                break;
-
-            case 'Jdbc':
-                $generatorJava.declareVariable(res, 'ipFinder',
-                    'org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinder');
-                $generatorJava.property(res, 'ipFinder', d.Jdbc, 'initSchema');
-
-                const datasource = d.Jdbc;
-                if (datasource.dataSourceBean && datasource.dialect) {
-                    res.needEmptyLine = !datasource.initSchema;
-
-                    res.line('ipFinder.setDataSource(DataSources.INSTANCE_' + datasource.dataSourceBean + ');');
-                }
-
-                res.needEmptyLine = true;
-
-                res.line('discovery.setIpFinder(ipFinder);');
-
-                break;
-
-            case 'SharedFs':
-                $generatorJava.beanProperty(res, 'discovery', d.SharedFs, 'ipFinder', 'ipFinder',
-                    'org.apache.ignite.spi.discovery.tcp.ipfinder.sharedfs.TcpDiscoverySharedFsIpFinder', {path: null}, true);
-
-                break;
-
-            case 'ZooKeeper':
-                const finderVar = 'ipFinder';
-
-                $generatorJava.declareVariable(res, 'ipFinder', 'org.apache.ignite.spi.discovery.tcp.ipfinder.zk.TcpDiscoveryZookeeperIpFinder');
-
-                if (d.ZooKeeper) {
-                    if ($generatorCommon.isDefinedAndNotEmpty(d.ZooKeeper.curator))
-                        res.line(finderVar + '.setCurator(new ' + res.importClass(d.ZooKeeper.curator) + '());');
-
-                    $generatorJava.property(res, finderVar, d.ZooKeeper, 'zkConnectionString');
-
-                    if (d.ZooKeeper.retryPolicy && d.ZooKeeper.retryPolicy.kind) {
-                        const kind = d.ZooKeeper.retryPolicy.kind;
-                        const retryPolicy = d.ZooKeeper.retryPolicy[kind];
-
-                        switch (kind) {
-                            case 'ExponentialBackoff':
-                                res.line(finderVar + '.setRetryPolicy(new ' + res.importClass('org.apache.curator.retry.ExponentialBackoffRetry') + '(' +
-                                    $generatorJava.constructorArg(retryPolicy, 'baseSleepTimeMs', 1000) +
-                                    $generatorJava.constructorArg(retryPolicy, 'maxRetries', 10, true) +
-                                    $generatorJava.constructorArg(retryPolicy, 'maxSleepMs', null, true, true) + '));');
-
-                                break;
-
-                            case 'BoundedExponentialBackoff':
-                                res.line(finderVar + '.setRetryPolicy(new ' + res.importClass('org.apache.curator.retry.BoundedExponentialBackoffRetry') + '(' +
-                                    $generatorJava.constructorArg(retryPolicy, 'baseSleepTimeMs', 1000) +
-                                    $generatorJava.constructorArg(retryPolicy, 'maxSleepTimeMs', 2147483647, true) +
-                                    $generatorJava.constructorArg(retryPolicy, 'maxRetries', 10, true) + '));');
-
-                                break;
-
-                            case 'UntilElapsed':
-                                res.line(finderVar + '.setRetryPolicy(new ' + res.importClass('org.apache.curator.retry.RetryUntilElapsed') + '(' +
-                                    $generatorJava.constructorArg(retryPolicy, 'maxElapsedTimeMs', 60000) +
-                                    $generatorJava.constructorArg(retryPolicy, 'sleepMsBetweenRetries', 1000, true) + '));');
-
-                                break;
-
-                            case 'NTimes':
-                                res.line(finderVar + '.setRetryPolicy(new ' + res.importClass('org.apache.curator.retry.RetryNTimes') + '(' +
-                                    $generatorJava.constructorArg(retryPolicy, 'n', 10) +
-                                    $generatorJava.constructorArg(retryPolicy, 'sleepMsBetweenRetries', 1000, true) + '));');
-
-                                break;
-
-                            case 'OneTime':
-                                res.line(finderVar + '.setRetryPolicy(new ' + res.importClass('org.apache.curator.retry.RetryOneTime') + '(' +
-                                    $generatorJava.constructorArg(retryPolicy, 'sleepMsBetweenRetry', 1000) + '));');
-
-                                break;
-
-                            case 'Forever':
-                                res.line(finderVar + '.setRetryPolicy(new ' + res.importClass('org.apache.curator.retry.RetryForever') + '(' +
-                                    $generatorJava.constructorArg(retryPolicy, 'retryIntervalMs', 1000) + '));');
-
-                                break;
-
-                            case 'Custom':
-                                if (retryPolicy && $generatorCommon.isDefinedAndNotEmpty(retryPolicy.className))
-                                    res.line(finderVar + '.setRetryPolicy(new ' + res.importClass(retryPolicy.className) + '());');
-
-                                break;
-
-                            default:
-                        }
-                    }
-
-                    $generatorJava.property(res, finderVar, d.ZooKeeper, 'basePath', null, null, '/services');
-                    $generatorJava.property(res, finderVar, d.ZooKeeper, 'serviceName', null, null, 'ignite');
-                    $generatorJava.property(res, finderVar, d.ZooKeeper, 'allowDuplicateRegistrations', null, null, false);
-                }
-
-                res.line('discovery.setIpFinder(ipFinder);');
-
-                break;
-
-            default:
-                res.line('Unknown discovery kind: ' + d.kind);
-        }
-
-        res.needEmptyLine = false;
-
-        $generatorJava.clusterDiscovery(d, res);
-
-        res.emptyLineIfNeeded();
-
-        res.line('cfg.setDiscoverySpi(discovery);');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate atomics group.
-$generatorJava.clusterAtomics = function(atomics, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.hasAtLeastOneProperty(atomics, ['cacheMode', 'atomicSequenceReserveSize', 'backups'])) {
-        res.startSafeBlock();
-
-        $generatorJava.declareVariable(res, 'atomicCfg', 'org.apache.ignite.configuration.AtomicConfiguration');
-
-        $generatorJava.enumProperty(res, 'atomicCfg', atomics, 'cacheMode', 'org.apache.ignite.cache.CacheMode', null, 'PARTITIONED');
-
-        const cacheMode = atomics.cacheMode ? atomics.cacheMode : 'PARTITIONED';
-
-        let hasData = cacheMode !== 'PARTITIONED';
-
-        hasData = $generatorJava.property(res, 'atomicCfg', atomics, 'atomicSequenceReserveSize', null, null, 1000) || hasData;
-
-        if (cacheMode === 'PARTITIONED')
-            hasData = $generatorJava.property(res, 'atomicCfg', atomics, 'backups', null, null, 0) || hasData;
-
-        res.needEmptyLine = true;
-
-        res.line('cfg.setAtomicConfiguration(atomicCfg);');
-
-        res.needEmptyLine = true;
-
-        if (!hasData)
-            res.rollbackSafeBlock();
-    }
-
-    return res;
-};
-
-// Generate binary group.
-$generatorJava.clusterBinary = function(binary, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.binaryIsDefined(binary)) {
-        const varName = 'binary';
-
-        $generatorJava.declareVariable(res, varName, 'org.apache.ignite.configuration.BinaryConfiguration');
-
-        if ($generatorCommon.isDefinedAndNotEmpty(binary.idMapper))
-            res.line(varName + '.setIdMapper(new ' + res.importClass(binary.idMapper) + '());');
-
-        if ($generatorCommon.isDefinedAndNotEmpty(binary.nameMapper))
-            res.line(varName + '.setNameMapper(new ' + res.importClass(binary.nameMapper) + '());');
-
-        if ($generatorCommon.isDefinedAndNotEmpty(binary.serializer))
-            res.line(varName + '.setSerializer(new ' + res.importClass(binary.serializer) + '());');
-
-        res.needEmptyLine = $generatorCommon.isDefinedAndNotEmpty(binary.idMapper) || $generatorCommon.isDefinedAndNotEmpty(binary.serializer);
-
-        if ($generatorCommon.isDefinedAndNotEmpty(binary.typeConfigurations)) {
-            const arrVar = 'types';
-
-            $generatorJava.declareVariable(res, arrVar, 'java.util.Collection', 'java.util.ArrayList', 'org.apache.ignite.binary.BinaryTypeConfiguration');
-
-            _.forEach(binary.typeConfigurations, function(type) {
-                if ($generatorCommon.isDefinedAndNotEmpty(type.typeName))
-                    res.line(arrVar + '.add(' + $generatorJava.binaryTypeFunctionName(type.typeName) + '());'); // TODO IGNITE-2269 Replace using of separated methods for binary type configurations to extended constructors.
-            });
-
-            res.needEmptyLine = true;
-
-            res.line(varName + '.setTypeConfigurations(' + arrVar + ');');
-
-            res.needEmptyLine = true;
-        }
-
-        $generatorJava.property(res, varName, binary, 'compactFooter', null, null, true);
-
-        res.needEmptyLine = true;
-
-        res.line('cfg.setBinaryConfiguration(' + varName + ');');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate cache key configurations.
-$generatorJava.clusterCacheKeyConfiguration = function(keyCfgs, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    keyCfgs = _.filter(keyCfgs, (cfg) => cfg.typeName && cfg.affinityKeyFieldName);
-
-    if (_.isEmpty(keyCfgs))
-        return res;
-
-    $generatorJava.declareVariableArray(res, 'keyConfigurations', 'org.apache.ignite.cache.CacheKeyConfiguration', keyCfgs.length);
-
-    const cacheKeyCfg = res.importClass('org.apache.ignite.cache.CacheKeyConfiguration');
-
-    _.forEach(keyCfgs, (cfg, idx) => {
-        res.needEmptyLine = true;
-
-        res.line(`keyConfigurations[${idx}] = new ${cacheKeyCfg}("${cfg.typeName}", "${cfg.affinityKeyFieldName}");`);
-
-        res.needEmptyLine = true;
-    });
-
-    res.line('cfg.setCacheKeyConfiguration(keyConfigurations);');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// TODO IGNITE-2269 Remove specified methods after implamentation of extended constructors.
-// Construct binary type configuration factory method name.
-$generatorJava.binaryTypeFunctionName = function(typeName) {
-    const dotIdx = typeName.lastIndexOf('.');
-
-    const shortName = dotIdx > 0 ? typeName.substr(dotIdx + 1) : typeName;
-
-    return $generatorCommon.toJavaName('binaryType', shortName);
-};
-
-// TODO IGNITE-2269 Remove specified methods after implamentation of extended constructors.
-// Generate factory method for specified BinaryTypeConfiguration.
-$generatorJava.binaryTypeConfiguration = function(type, res) {
-    const typeName = type.typeName;
-
-    res.line('/**');
-    res.line(' * Create binary type configuration for ' + typeName + '.');
-    res.line(' *');
-    res.line(' * @return Configured binary type.');
-    res.line(' */');
-    res.startBlock('private static BinaryTypeConfiguration ' + $generatorJava.binaryTypeFunctionName(typeName) + '() {');
-
-    $generatorJava.resetVariables(res);
-
-    const typeVar = 'typeCfg';
-
-    $generatorJava.declareVariable(res, typeVar, 'org.apache.ignite.binary.BinaryTypeConfiguration');
-
-    $generatorJava.property(res, typeVar, type, 'typeName');
-
-    if ($generatorCommon.isDefinedAndNotEmpty(type.idMapper))
-        res.line(typeVar + '.setIdMapper(new ' + res.importClass(type.idMapper) + '());');
-
-    if ($generatorCommon.isDefinedAndNotEmpty(type.nameMapper))
-        res.line(typeVar + '.setNameMapper(new ' + res.importClass(type.nameMapper) + '());');
-
-    if ($generatorCommon.isDefinedAndNotEmpty(type.serializer))
-        res.line(typeVar + '.setSerializer(new ' + res.importClass(type.serializer) + '());');
-
-    $generatorJava.property(res, typeVar, type, 'enum', null, null, false);
-
-    res.needEmptyLine = true;
-
-    res.line('return ' + typeVar + ';');
-    res.endBlock('}');
-
-    res.needEmptyLine = true;
-};
-
-// TODO IGNITE-2269 Remove specified methods after implamentation of extended constructors.
-// Generates binary type configuration factory methods.
-$generatorJava.binaryTypeConfigurations = function(binary, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!_.isNil(binary)) {
-        _.forEach(binary.typeConfigurations, function(type) {
-            $generatorJava.binaryTypeConfiguration(type, res);
-        });
-    }
-
-    return res;
-};
-
-// Generate collision group.
-$generatorJava.clusterCollision = function(collision, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (collision && collision.kind && collision.kind !== 'Noop') {
-        const spi = collision[collision.kind];
-
-        if (collision.kind !== 'Custom' || (spi && $generatorCommon.isDefinedAndNotEmpty(spi.class))) {
-            const varName = 'collisionSpi';
-
-            switch (collision.kind) {
-                case 'JobStealing':
-                    $generatorJava.declareVariable(res, varName, 'org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi');
-
-                    $generatorJava.property(res, varName, spi, 'activeJobsThreshold', null, null, 95);
-                    $generatorJava.property(res, varName, spi, 'waitJobsThreshold', null, null, 0);
-                    $generatorJava.property(res, varName, spi, 'messageExpireTime', null, null, 1000);
-                    $generatorJava.property(res, varName, spi, 'maximumStealingAttempts', null, null, 5);
-                    $generatorJava.property(res, varName, spi, 'stealingEnabled', null, null, true);
-
-                    if ($generatorCommon.isDefinedAndNotEmpty(spi.externalCollisionListener)) {
-                        res.line(varName + '.' + $generatorJava.setterName('externalCollisionListener') +
-                            '(new ' + res.importClass(spi.externalCollisionListener) + '());');
-                    }
-
-                    if ($generatorCommon.isDefinedAndNotEmpty(spi.stealingAttributes)) {
-                        const stealingAttrsVar = 'stealingAttrs';
-
-                        res.needEmptyLine = true;
-
-                        $generatorJava.declareVariable(res, stealingAttrsVar, 'java.util.Map', 'java.util.HashMap', 'String', 'java.io.Serializable');
-
-                        _.forEach(spi.stealingAttributes, function(attr) {
-                            res.line(stealingAttrsVar + '.put("' + attr.name + '", "' + attr.value + '");');
-                        });
-
-                        res.needEmptyLine = true;
-
-                        res.line(varName + '.setStealingAttributes(' + stealingAttrsVar + ');');
-                    }
-
-                    break;
-
-                case 'FifoQueue':
-                    $generatorJava.declareVariable(res, varName, 'org.apache.ignite.spi.collision.fifoqueue.FifoQueueCollisionSpi');
-
-                    $generatorJava.property(res, varName, spi, 'parallelJobsNumber');
-                    $generatorJava.property(res, varName, spi, 'waitingJobsNumber');
-
-                    break;
-
-                case 'PriorityQueue':
-                    $generatorJava.declareVariable(res, varName, 'org.apache.ignite.spi.collision.priorityqueue.PriorityQueueCollisionSpi');
-
-                    $generatorJava.property(res, varName, spi, 'parallelJobsNumber');
-                    $generatorJava.property(res, varName, spi, 'waitingJobsNumber');
-                    $generatorJava.property(res, varName, spi, 'priorityAttributeKey', null, null, 'grid.task.priority');
-                    $generatorJava.property(res, varName, spi, 'jobPriorityAttributeKey', null, null, 'grid.job.priority');
-                    $generatorJava.property(res, varName, spi, 'defaultPriority', null, null, 0);
-                    $generatorJava.property(res, varName, spi, 'starvationIncrement', null, null, 1);
-                    $generatorJava.property(res, varName, spi, 'starvationPreventionEnabled', null, null, true);
-
-                    break;
-
-                case 'Custom':
-                    $generatorJava.declareVariable(res, varName, spi.class);
-
-                    break;
-
-                default:
-            }
-
-            res.needEmptyLine = true;
-
-            res.line('cfg.setCollisionSpi(' + varName + ');');
-
-            res.needEmptyLine = true;
-        }
-    }
-
-    return res;
-};
-
-// Generate communication group.
-$generatorJava.clusterCommunication = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    const cfg = $generatorCommon.COMMUNICATION_CONFIGURATION;
-
-    $generatorJava.beanProperty(res, 'cfg', cluster.communication, 'communicationSpi', 'commSpi', cfg.className, cfg.fields);
-
-    res.needEmptyLine = false;
-
-    $generatorJava.property(res, 'cfg', cluster, 'networkTimeout', null, null, 5000);
-    $generatorJava.property(res, 'cfg', cluster, 'networkSendRetryDelay', null, null, 1000);
-    $generatorJava.property(res, 'cfg', cluster, 'networkSendRetryCount', null, null, 3);
-    $generatorJava.property(res, 'cfg', cluster, 'segmentCheckFrequency');
-    $generatorJava.property(res, 'cfg', cluster, 'waitForSegmentOnStart', null, null, false);
-    $generatorJava.property(res, 'cfg', cluster, 'discoveryStartupDelay', null, null, 60000);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate REST access group.
-$generatorJava.clusterConnector = function(connector, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!_.isNil(connector) && connector.enabled) {
-        const cfg = _.cloneDeep($generatorCommon.CONNECTOR_CONFIGURATION);
-
-        if (connector.sslEnabled) {
-            cfg.fields.sslClientAuth = {dflt: false};
-            cfg.fields.sslFactory = {type: 'bean'};
-        }
-
-        $generatorJava.beanProperty(res, 'cfg', connector, 'connectorConfiguration', 'clientCfg',
-            cfg.className, cfg.fields, true);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate deployment group.
-$generatorJava.clusterDeployment = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorJava.enumProperty(res, 'cfg', cluster, 'deploymentMode', 'org.apache.ignite.configuration.DeploymentMode', null, 'SHARED');
-
-    res.softEmptyLine();
-
-    const p2pEnabled = cluster.peerClassLoadingEnabled;
-
-    if (!_.isNil(p2pEnabled)) {
-        $generatorJava.property(res, 'cfg', cluster, 'peerClassLoadingEnabled', null, null, false);
-
-        if (p2pEnabled) {
-            $generatorJava.property(res, 'cfg', cluster, 'peerClassLoadingMissedResourcesCacheSize', null, null, 100);
-            $generatorJava.property(res, 'cfg', cluster, 'peerClassLoadingThreadPoolSize', null, null, 2);
-            $generatorJava.multiparamProperty(res, 'cfg', cluster, 'peerClassLoadingLocalClassPathExclude');
-        }
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate discovery group.
-$generatorJava.clusterDiscovery = function(disco, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (disco) {
-        $generatorJava.property(res, 'discovery', disco, 'localAddress');
-        $generatorJava.property(res, 'discovery', disco, 'localPort', null, null, 47500);
-        $generatorJava.property(res, 'discovery', disco, 'localPortRange', null, null, 100);
-
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.addressResolver)) {
-            $generatorJava.beanProperty(res, 'discovery', disco, 'addressResolver', 'addressResolver', disco.addressResolver, {}, true);
-            res.needEmptyLine = false;
-        }
-
-        $generatorJava.property(res, 'discovery', disco, 'socketTimeout', null, null, 5000);
-        $generatorJava.property(res, 'discovery', disco, 'ackTimeout', null, null, 5000);
-        $generatorJava.property(res, 'discovery', disco, 'maxAckTimeout', null, null, 600000);
-        $generatorJava.property(res, 'discovery', disco, 'networkTimeout', null, null, 5000);
-        $generatorJava.property(res, 'discovery', disco, 'joinTimeout', null, null, 0);
-        $generatorJava.property(res, 'discovery', disco, 'threadPriority', null, null, 10);
-        $generatorJava.property(res, 'discovery', disco, 'heartbeatFrequency', null, null, 2000);
-        $generatorJava.property(res, 'discovery', disco, 'maxMissedHeartbeats', null, null, 1);
-        $generatorJava.property(res, 'discovery', disco, 'maxMissedClientHeartbeats', null, null, 5);
-        $generatorJava.property(res, 'discovery', disco, 'topHistorySize', null, null, 1000);
-
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.listener)) {
-            $generatorJava.beanProperty(res, 'discovery', disco, 'listener', 'listener', disco.listener, {}, true);
-            res.needEmptyLine = false;
-        }
-
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.dataExchange)) {
-            $generatorJava.beanProperty(res, 'discovery', disco, 'dataExchange', 'dataExchange', disco.dataExchange, {}, true);
-            res.needEmptyLine = false;
-        }
-
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.metricsProvider)) {
-            $generatorJava.beanProperty(res, 'discovery', disco, 'metricsProvider', 'metricsProvider', disco.metricsProvider, {}, true);
-            res.needEmptyLine = false;
-        }
-
-        $generatorJava.property(res, 'discovery', disco, 'reconnectCount', null, null, 10);
-        $generatorJava.property(res, 'discovery', disco, 'statisticsPrintFrequency', null, null, 0);
-        $generatorJava.property(res, 'discovery', disco, 'ipFinderCleanFrequency', null, null, 60000);
-
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.authenticator)) {
-            $generatorJava.beanProperty(res, 'discovery', disco, 'authenticator', 'authenticator', disco.authenticator, {}, true);
-            res.needEmptyLine = false;
-        }
-
-        $generatorJava.property(res, 'discovery', disco, 'forceServerMode', null, null, false);
-        $generatorJava.property(res, 'discovery', disco, 'clientReconnectDisabled', null, null, false);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate events group.
-$generatorJava.clusterEvents = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (cluster.includeEventTypes && cluster.includeEventTypes.length > 0) {
-        res.emptyLineIfNeeded();
-
-        const evtGrps = angular.element(document.getElementById('app')).injector().get('igniteEventGroups');
-
-        if (cluster.includeEventTypes.length === 1) {
-            const evtGrp = _.find(evtGrps, {value: cluster.includeEventTypes[0]});
-            const evts = res.importStatic(evtGrp.class + '.' + evtGrp.value);
-
-            res.line('cfg.setIncludeEventTypes(' + evts + ');');
-        }
-        else {
-            _.forEach(cluster.includeEventTypes, function(value, ix) {
-                const evtGrp = _.find(evtGrps, {value});
-                const evts = res.importStatic(evtGrp.class + '.' + evtGrp.value);
-
-                if (ix === 0)
-                    res.line('int[] events = new int[' + evts + '.length');
-                else
-                    res.line('    + ' + evts + '.length');
-            });
-
-            res.line('];');
-
-            res.needEmptyLine = true;
-
-            res.line('int k = 0;');
-
-            _.forEach(cluster.includeEventTypes, function(value, idx) {
-                res.needEmptyLine = true;
-
-                const evtGrp = _.find(evtGrps, {value});
-                const evts = res.importStatic(evtGrp.class + '.' + value);
-
-                res.line('System.arraycopy(' + evts + ', 0, events, k, ' + evts + '.length);');
-
-                if (idx < cluster.includeEventTypes.length - 1)
-                    res.line('k += ' + evts + '.length;');
-            });
-
-            res.needEmptyLine = true;
-
-            res.line('cfg.setIncludeEventTypes(events);');
-        }
-
-        res.needEmptyLine = true;
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate failover group.
-$generatorJava.clusterFailover = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.isDefinedAndNotEmpty(cluster.failoverSpi) && _.findIndex(cluster.failoverSpi, function(spi) {
-        return $generatorCommon.isDefinedAndNotEmpty(spi.kind) && (spi.kind !== 'Custom' || $generatorCommon.isDefinedAndNotEmpty(_.get(spi, spi.kind + '.class')));
-    }) >= 0) {
-        const arrayVarName = 'failoverSpiList';
-
-        $generatorJava.declareVariable(res, arrayVarName, 'java.util.List', 'java.util.ArrayList', 'org.apache.ignite.spi.failover.FailoverSpi');
-
-        _.forEach(cluster.failoverSpi, function(spi) {
-            if (spi.kind && (spi.kind !== 'Custom' || $generatorCommon.isDefinedAndNotEmpty(_.get(spi, spi.kind + '.class')))) {
-                const varName = 'failoverSpi';
-
-                const maxAttempts = _.get(spi, spi.kind + '.maximumFailoverAttempts');
-
-                if ((spi.kind === 'JobStealing' || spi.kind === 'Always') && $generatorCommon.isDefinedAndNotEmpty(maxAttempts) && maxAttempts !== 5) {
-                    const spiCls = res.importClass($generatorCommon.failoverSpiClass(spi));
-
-                    $generatorJava.declareVariableCustom(res, varName, 'org.apache.ignite.spi.failover.FailoverSpi', 'new ' + spiCls + '()');
-
-                    if ($generatorCommon.isDefinedAndNotEmpty(spi[spi.kind].maximumFailoverAttempts))
-                        res.line('((' + spiCls + ') ' + varName + ').setMaximumFailoverAttempts(' + spi[spi.kind].maximumFailoverAttempts + ');');
-
-                    res.needEmptyLine = true;
-
-                    res.line(arrayVarName + '.add(' + varName + ');');
-                }
-                else
-                    res.line(arrayVarName + '.add(new ' + res.importClass($generatorCommon.failoverSpiClass(spi)) + '());');
-
-                res.needEmptyLine = true;
-            }
-        });
-
-        res.line('cfg.setFailoverSpi(' + arrayVarName + '.toArray(new FailoverSpi[' + arrayVarName + '.size()]));');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate marshaller group.
-$generatorJava.clusterLogger = function(logger, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.loggerConfigured(logger)) {
-        const varName = 'logger';
-
-        const log = logger[logger.kind];
-
-        switch (logger.kind) {
-            case 'Log4j2':
-                $generatorJava.declareVariableCustom(res, varName, 'org.apache.ignite.logger.log4j2.Log4J2Logger',
-                    'new Log4J2Logger(' + $generatorJava.toJavaCode(log.path, 'path') + ')');
-
-                res.needEmptyLine = true;
-
-                if ($generatorCommon.isDefinedAndNotEmpty(log.level))
-                    res.line(varName + '.setLevel(' + res.importClass('org.apache.logging.log4j.Level') + '.' + log.level + ');');
-
-                break;
-
-            case 'Null':
-                $generatorJava.declareVariable(res, varName, 'org.apache.ignite.logger.NullLogger');
-
-                break;
-
-            case 'Java':
-                $generatorJava.declareVariable(res, varName, 'org.apache.ignite.logger.java.JavaLogger');
-
-                break;
-
-            case 'JCL':
-                $generatorJava.declareVariable(res, varName, 'org.apache.ignite.logger.jcl.JclLogger');
-
-                break;
-
-            case 'SLF4J':
-                $generatorJava.declareVariable(res, varName, 'org.apache.ignite.logger.slf4j.Slf4jLogger');
-
-                break;
-
-            case 'Log4j':
-                if (log.mode === 'Default')
-                    $generatorJava.declareVariable(res, varName, 'org.apache.ignite.logger.log4j.Log4JLogger');
-                else {
-                    $generatorJava.declareVariableCustom(res, varName, 'org.apache.ignite.logger.log4j.Log4JLogger',
-                        'new Log4JLogger(' + $generatorJava.toJavaCode(log.path, 'path') + ')');
-                }
-
-                if ($generatorCommon.isDefinedAndNotEmpty(log.level))
-                    res.line(varName + '.setLevel(' + res.importClass('org.apache.log4j.Level') + '.' + log.level + ');');
-
-                break;
-
-            case 'Custom':
-                $generatorJava.declareVariable(res, varName, log.class);
-
-                break;
-
-            default:
-        }
-
-        res.needEmptyLine = true;
-
-        res.line('cfg.setGridLogger(' + varName + ');');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate marshaller group.
-$generatorJava.clusterMarshaller = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    const marshaller = cluster.marshaller;
-
-    if (marshaller && marshaller.kind) {
-        const marshallerDesc = $generatorCommon.MARSHALLERS[marshaller.kind];
-
-        $generatorJava.beanProperty(res, 'cfg', marshaller[marshaller.kind], 'marshaller', 'marshaller',
-            marshallerDesc.className, marshallerDesc.fields, true);
-
-        $generatorJava.beanProperty(res, 'marshaller', marshaller[marshaller.kind], marshallerDesc.className, marshallerDesc.fields, true);
-    }
-
-    $generatorJava.property(res, 'cfg', cluster, 'marshalLocalJobs', null, null, false);
-    $generatorJava.property(res, 'cfg', cluster, 'marshallerCacheKeepAliveTime', null, null, 10000);
-    $generatorJava.property(res, 'cfg', cluster, 'marshallerCacheThreadPoolSize', null, 'setMarshallerCachePoolSize');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate metrics group.
-$generatorJava.clusterMetrics = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorJava.property(res, 'cfg', cluster, 'metricsExpireTime');
-    $generatorJava.property(res, 'cfg', cluster, 'metricsHistorySize', null, null, 10000);
-    $generatorJava.property(res, 'cfg', cluster, 'metricsLogFrequency', null, null, 60000);
-    $generatorJava.property(res, 'cfg', cluster, 'metricsUpdateFrequency', null, null, 2000);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate swap group.
-$generatorJava.clusterSwap = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (cluster.swapSpaceSpi && cluster.swapSpaceSpi.kind === 'FileSwapSpaceSpi') {
-        $generatorJava.beanProperty(res, 'cfg', cluster.swapSpaceSpi.FileSwapSpaceSpi, 'swapSpaceSpi', 'swapSpi',
-            $generatorCommon.SWAP_SPACE_SPI.className, $generatorCommon.SWAP_SPACE_SPI.fields, true);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate time group.
-$generatorJava.clusterTime = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorJava.property(res, 'cfg', cluster, 'clockSyncSamples', null, null, 8);
-    $generatorJava.property(res, 'cfg', cluster, 'clockSyncFrequency', null, null, 120000);
-    $generatorJava.property(res, 'cfg', cluster, 'timeServerPortBase', null, null, 31100);
-    $generatorJava.property(res, 'cfg', cluster, 'timeServerPortRange', null, null, 100);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate ODBC configuration group.
-$generatorJava.clusterODBC = function(odbc, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (odbc && odbc.odbcEnabled) {
-        $generatorJava.beanProperty(res, 'cfg', odbc, 'odbcConfiguration', 'odbcConfiguration',
-            $generatorCommon.ODBC_CONFIGURATION.className, $generatorCommon.ODBC_CONFIGURATION.fields, true);
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate thread pools group.
-$generatorJava.clusterPools = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorJava.property(res, 'cfg', cluster, 'publicThreadPoolSize');
-    $generatorJava.property(res, 'cfg', cluster, 'systemThreadPoolSize');
-    $generatorJava.property(res, 'cfg', cluster, 'managementThreadPoolSize');
-    $generatorJava.property(res, 'cfg', cluster, 'igfsThreadPoolSize');
-    $generatorJava.property(res, 'cfg', cluster, 'rebalanceThreadPoolSize');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate transactions group.
-$generatorJava.clusterTransactions = function(transactionConfiguration, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorJava.beanProperty(res, 'cfg', transactionConfiguration, 'transactionConfiguration',
-        'transactionConfiguration', $generatorCommon.TRANSACTION_CONFIGURATION.className,
-        $generatorCommon.TRANSACTION_CONFIGURATION.fields, false);
-
-    return res;
-};
-
-// Generate user attributes group.
-$generatorJava.clusterUserAttributes = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.isDefinedAndNotEmpty(cluster.attributes)) {
-        $generatorJava.declareVariable(res, 'attributes', 'java.util.Map', 'java.util.HashMap', 'java.lang.String', 'java.lang.String');
-
-        _.forEach(cluster.attributes, function(attr) {
-            res.line('attributes.put("' + attr.name + '", "' + attr.value + '");');
-        });
-
-        res.needEmptyLine = true;
-
-        res.line('cfg.setUserAttributes(attributes);');
-
-        res.needEmptyLine = true;
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-
-// Generate cache general group.
-$generatorJava.cacheGeneral = function(cache, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!varName)
-        varName = $generatorJava.nextVariableName('cache', cache);
-
-    $generatorJava.property(res, varName, cache, 'name');
-
-    $generatorJava.enumProperty(res, varName, cache, 'cacheMode', 'org.apache.ignite.cache.CacheMode');
-    $generatorJava.enumProperty(res, varName, cache, 'atomicityMode', 'org.apache.ignite.cache.CacheAtomicityMode');
-
-    if (cache.cacheMode === 'PARTITIONED' && $generatorJava.property(res, varName, cache, 'backups'))
-        $generatorJava.property(res, varName, cache, 'readFromBackup');
-
-    $generatorJava.property(res, varName, cache, 'copyOnRead');
-
-    if (cache.cacheMode === 'PARTITIONED' && cache.atomicityMode === 'TRANSACTIONAL')
-        $generatorJava.property(res, varName, cache, 'invalidate');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache memory group.
-$generatorJava.cacheMemory = function(cache, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!varName)
-        varName = $generatorJava.nextVariableName('cache', cache);
-
-    $generatorJava.enumProperty(res, varName, cache, 'memoryMode', 'org.apache.ignite.cache.CacheMemoryMode', null, 'ONHEAP_TIERED');
-
-    if (cache.memoryMode !== 'OFFHEAP_VALUES')
-        $generatorJava.property(res, varName, cache, 'offHeapMaxMemory', null, null, -1);
-
-    res.softEmptyLine();
-
-    $generatorJava.evictionPolicy(res, varName, cache.evictionPolicy, 'evictionPolicy');
-
-    $generatorJava.property(res, varName, cache, 'startSize', null, null, 1500000);
-    $generatorJava.property(res, varName, cache, 'swapEnabled', null, null, false);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache query & indexing group.
-$generatorJava.cacheQuery = function(cache, domains, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!varName)
-        varName = $generatorJava.nextVariableName('cache', cache);
-
-    $generatorJava.property(res, varName, cache, 'sqlSchema');
-    $generatorJava.property(res, varName, cache, 'sqlOnheapRowCacheSize', null, null, 10240);
-    $generatorJava.property(res, varName, cache, 'longQueryWarningTimeout', null, null, 3000);
-
-    const indexedTypes = _.reduce(domains, (acc, domain) => {
-        if (domain.queryMetadata === 'Annotations') {
-            acc.push(domain.keyType);
-            acc.push(domain.valueType);
-        }
-
-        return acc;
-    }, []);
-
-    if (indexedTypes.length > 0) {
-        res.softEmptyLine();
-
-        $generatorJava.multiparamProperty(res, varName, {indexedTypes}, 'indexedTypes', 'class');
-    }
-
-    res.softEmptyLine();
-
-    $generatorJava.multiparamProperty(res, varName, cache, 'sqlFunctionClasses', 'class');
-
-    res.softEmptyLine();
-
-    $generatorJava.property(res, varName, cache, 'snapshotableIndex', null, null, false);
-    $generatorJava.property(res, varName, cache, 'sqlEscapeAll', null, null, false);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-/**
- * Generate cache store datasource.
- *
- * @param storeFactory Factory to generate data source for.
- * @param res Resulting output with generated code.
- */
-$generatorJava.cacheStoreDataSource = function(storeFactory, res) {
-    const dialect = storeFactory.connectVia ? (storeFactory.connectVia === 'DataSource' ? storeFactory.dialect : null) : storeFactory.dialect;
-
-    if (dialect) {
-        const varName = 'dataSource';
-
-        const dataSourceBean = storeFactory.dataSourceBean;
-
-        const varType = res.importClass($generatorCommon.dataSourceClassName(dialect));
-
-        res.line('public static final ' + varType + ' INSTANCE_' + dataSourceBean + ' = create' + dataSourceBean + '();');
-
-        res.needEmptyLine = true;
-
-        res.startBlock('private static ' + varType + ' create' + dataSourceBean + '() {');
-        if (dialect === 'Oracle')
-            res.startBlock('try {');
-
-        $generatorJava.resetVariables(res);
-
-        $generatorJava.declareVariable(res, varName, varType);
-
-        switch (dialect) {
-            case 'Generic':
-                res.line(varName + '.setJdbcUrl(props.getProperty("' + dataSourceBean + '.jdbc.url"));');
-
-                break;
-
-            case 'DB2':
-                res.line(varName + '.setServerName(props.getProperty("' + dataSourceBean + '.jdbc.server_name"));');
-                res.line(varName + '.setPortNumber(Integer.valueOf(props.getProperty("' + dataSourceBean + '.jdbc.port_number")));');
-                res.line(varName + '.setDatabaseName(props.getProperty("' + dataSourceBean + '.jdbc.database_name"));');
-                res.line(varName + '.setDriverType(Integer.valueOf(props.getProperty("' + dataSourceBean + '.jdbc.driver_type")));');
-
-                break;
-
-            case 'PostgreSQL':
-                res.line(varName + '.setUrl(props.getProperty("' + dataSourceBean + '.jdbc.url"));');
-
-                break;
-
-            default:
-                res.line(varName + '.setURL(props.getProperty("' + dataSourceBean + '.jdbc.url"));');
-        }
-
-        res.line(varName + '.setUser(props.getProperty("' + dataSourceBean + '.jdbc.username"));');
-        res.line(varName + '.setPassword(props.getProperty("' + dataSourceBean + '.jdbc.password"));');
-
-        res.needEmptyLine = true;
-
-        res.line('return dataSource;');
-
-        if (dialect === 'Oracle') {
-            res.endBlock('}');
-            res.startBlock('catch (' + res.importClass('java.sql.SQLException') + ' ex) {');
-            res.line('throw new Error(ex);');
-            res.endBlock('}');
-        }
-
-        res.endBlock('}');
-
-        res.needEmptyLine = true;
-
-        return dataSourceBean;
-    }
-
-    return null;
-};
-
-$generatorJava.clusterDataSources = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    const datasources = [];
-
-    let storeFound = false;
-
-    function startSourcesFunction() {
-        if (!storeFound) {
-            res.line('/** Helper class for datasource creation. */');
-            res.startBlock('public static class DataSources {');
-
-            storeFound = true;
-        }
-    }
-
-    _.forEach(cluster.caches, function(cache) {
-        const factoryKind = cache.cacheStoreFactory.kind;
-
-        const storeFactory = cache.cacheStoreFactory[factoryKind];
-
-        if (storeFactory) {
-            const beanClassName = $generatorJava.dataSourceClassName(res, storeFactory);
-
-            if (beanClassName && !_.includes(datasources, beanClassName)) {
-                datasources.push(beanClassName);
-
-                if (factoryKind === 'CacheJdbcPojoStoreFactory' || factoryKind === 'CacheJdbcBlobStoreFactory') {
-                    startSourcesFunction();
-
-                    $generatorJava.cacheStoreDataSource(storeFactory, res);
-                }
-            }
-        }
-    });
-
-    if (cluster.discovery.kind === 'Jdbc') {
-        const datasource = cluster.discovery.Jdbc;
-
-        if (datasource.dataSourceBean && datasource.dialect) {
-            const beanClassName = $generatorJava.dataSourceClassName(res, datasource);
-
-            if (beanClassName && !_.includes(datasources, beanClassName)) {
-                startSourcesFunction();
-
-                $generatorJava.cacheStoreDataSource(datasource, res);
-            }
-        }
-    }
-
-    if (storeFound)
-        res.endBlock('}');
-
-    return res;
-};
-
-/**
- * Generate cache store group.
- *
- * @param cache Cache descriptor.
- * @param domains Domain model descriptors.
- * @param cacheVarName Cache variable name.
- * @param res Resulting output with generated code.
- * @returns {*} Java code for cache store configuration.
- */
-$generatorJava.cacheStore = function(cache, domains, cacheVarName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!cacheVarName)
-        cacheVarName = $generatorJava.nextVariableName('cache', cache);
-
-    if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) {
-        const factoryKind = cache.cacheStoreFactory.kind;
-
-        const storeFactory = cache.cacheStoreFactory[factoryKind];
-
-        if (storeFactory) {
-            const storeFactoryDesc = $generatorCommon.STORE_FACTORIES[factoryKind];
-
-            const varName = 'storeFactory' + storeFactoryDesc.suffix;
-
-            if (factoryKind === 'CacheJdbcPojoStoreFactory') {
-                // Generate POJO store factory.
-                $generatorJava.declareVariable(res, varName, 'org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory', null, null, null, true);
-                res.deep++;
-
-                res.line('/** {@inheritDoc} */');
-                res.startBlock('@Override public ' + res.importClass('org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStore') + ' create() {');
-
-                res.line('setDataSource(DataSources.INSTANCE_' + storeFactory.dataSourceBean + ');');
-
-                res.needEmptyLine = true;
-
-                res.line('return super.create();');
-                res.endBlock('}');
-                res.endBlock('};');
-
-                res.needEmptyLine = true;
-
-                res.line(varName + '.setDialect(new ' +
-                    res.importClass($generatorCommon.jdbcDialectClassName(storeFactory.dialect)) + '());');
-
-                res.needEmptyLine = true;
-
-                if (storeFactory.sqlEscapeAll) {
-                    res.line(varName + '.setSqlEscapeAll(true);');
-
-                    res.needEmptyLine = true;
-                }
-
-                const domainConfigs = _.filter(domains, function(domain) {
-                    return $generatorCommon.domainQueryMetadata(domain) === 'Configuration' &&
-                        $generatorCommon.isDefinedAndNotEmpty(domain.databaseTable);
-                });
-
-                if ($generatorCommon.isDefinedAndNotEmpty(domainConfigs)) {
-                    $generatorJava.declareVariable(res, 'jdbcTypes', 'java.util.Collection', 'java.util.ArrayList', 'org.apache.ignite.cache.store.jdbc.JdbcType');
-
-                    res.needEmptyLine = true;
-
-                    _.forEach(domainConfigs, function(domain) {
-                        if ($generatorCommon.isDefinedAndNotEmpty(domain.databaseTable))
-                            res.line('jdbcTypes.add(jdbcType' + $generatorJava.extractType(domain.valueType) + '(' + cacheVarName + '.getName()));');
-                    });
-
-                    res.needEmptyLine = true;
-
-                    res.line(varName + '.setTypes(jdbcTypes.toArray(new JdbcType[jdbcTypes.size()]));');
-
-                    res.needEmptyLine = true;
-                }
-
-                res.line(cacheVarName + '.setCacheStoreFactory(' + varName + ');');
-            }
-            else if (factoryKind === 'CacheJdbcBlobStoreFactory') {
-                // Generate POJO store factory.
-                $generatorJava.declareVariable(res, varName, 'org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactory', null, null, null, storeFactory.connectVia === 'DataSource');
-
-                if (storeFactory.connectVia === 'DataSource') {
-                    res.deep++;
-
-                    res.line('/** {@inheritDoc} */');
-                    res.startBlock('@Override public ' + res.importClass('org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStore') + ' create() {');
-
-                    res.line('setDataSource(DataSources.INSTANCE_' + storeFactory.dataSourceBean + ');');
-
-                    res.needEmptyLine = true;
-
-                    res.line('return super.create();');
-                    res.endBlock('}');
-                    res.endBlock('};');
-
-                    res.needEmptyLine = true;
-
-                    $generatorJava.property(res, varName, storeFactory, 'initSchema');
-                    $generatorJava.property(res, varName, storeFactory, 'createTableQuery');
-                    $generatorJava.property(res, varName, storeFactory, 'loadQuery');
-                    $generatorJava.property(res, varName, storeFactory, 'insertQuery');
-                    $generatorJava.property(res, varName, storeFactory, 'updateQuery');
-                    $generatorJava.property(res, varName, storeFactory, 'deleteQuery');
-                }
-                else {
-                    $generatorJava.property(res, varName, storeFactory, 'connectionUrl');
-
-                    if (storeFactory.user) {
-                        $generatorJava.property(res, varName, storeFactory, 'user');
-                        res.line(varName + '.setPassword(props.getProperty("ds.' + storeFactory.user + '.password"));');
-                    }
-                }
-
-                res.needEmptyLine = true;
-
-                res.line(cacheVarName + '.setCacheStoreFactory(' + varName + ');');
-            }
-            else
-                $generatorJava.beanProperty(res, cacheVarName, storeFactory, 'cacheStoreFactory', varName, storeFactoryDesc.className, storeFactoryDesc.fields, true);
-
-            res.needEmptyLine = true;
-        }
-    }
-
-    res.softEmptyLine();
-
-    $generatorJava.property(res, cacheVarName, cache, 'storeKeepBinary', null, null, false);
-    $generatorJava.property(res, cacheVarName, cache, 'loadPreviousValue', null, null, false);
-    $generatorJava.property(res, cacheVarName, cache, 'readThrough', null, null, false);
-    $generatorJava.property(res, cacheVarName, cache, 'writeThrough', null, null, false);
-
-    res.softEmptyLine();
-
-    if (cache.writeBehindEnabled) {
-        $generatorJava.property(res, cacheVarName, cache, 'writeBehindEnabled', null, null, false);
-        $generatorJava.property(res, cacheVarName, cache, 'writeBehindBatchSize', null, null, 512);
-        $generatorJava.property(res, cacheVarName, cache, 'writeBehindFlushSize', null, null, 10240);
-        $generatorJava.property(res, cacheVarName, cache, 'writeBehindFlushFrequency', null, null, 5000);
-        $generatorJava.property(res, cacheVarName, cache, 'writeBehindFlushThreadCount', null, null, 1);
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache node filter group.
-$generatorJava.cacheNodeFilter = function(cache, igfss, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!varName)
-        varName = $generatorJava.nextVariableName('cache', cache);
-
-    switch (_.get(cache, 'nodeFilter.kind')) {
-        case 'IGFS':
-            const foundIgfs = _.find(igfss, (igfs) => igfs._id === cache.nodeFilter.IGFS.igfs);
-
-            if (foundIgfs) {
-                const predClsName = res.importClass('org.apache.ignite.internal.processors.igfs.IgfsNodePredicate');
-
-                res.line(`${varName}.setNodeFilter(new ${predClsName}("${foundIgfs.name}"));`);
-            }
-
-            break;
-
-        case 'OnNodes':
-            const nodes = cache.nodeFilter.OnNodes.nodeIds;
-
-            if ($generatorCommon.isDefinedAndNotEmpty(nodes)) {
-                const startQuote = res.importClass('java.util.UUID') + '.fromString("';
-
-                $generatorJava.fxVarArgs(res, varName + '.setNodeFilter(new ' +
-                    res.importClass('org.apache.ignite.internal.util.lang.GridNodePredicate'), true, nodes, '(', '))',
-                    startQuote, '")');
-            }
-
-            break;
-
-        case 'Custom':
-            res.line(varName + '.setNodeFilter(new ' + res.importClass(cache.nodeFilter.Custom.className) + '());');
-
-            break;
-
-        default: break;
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache concurrency group.
-$generatorJava.cacheConcurrency = function(cache, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!varName)
-        varName = $generatorJava.nextVariableName('cache', cache);
-
-    $generatorJava.property(res, varName, cache, 'maxConcurrentAsyncOperations', null, null, 500);
-    $generatorJava.property(res, varName, cache, 'defaultLockTimeout', null, null, 0);
-    $generatorJava.enumProperty(res, varName, cache, 'atomicWriteOrderMode', 'org.apache.ignite.cache.CacheAtomicWriteOrderMode');
-    $generatorJava.enumProperty(res, varName, cache, 'writeSynchronizationMode', 'org.apache.ignite.cache.CacheWriteSynchronizationMode', null, 'PRIMARY_SYNC');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache rebalance group.
-$generatorJava.cacheRebalance = function(cache, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!varName)
-        varName = $generatorJava.nextVariableName('cache', cache);
-
-    if (cache.cacheMode !== 'LOCAL') {
-        $generatorJava.enumProperty(res, varName, cache, 'rebalanceMode', 'org.apache.ignite.cache.CacheRebalanceMode', null, 'ASYNC');
-        $generatorJava.property(res, varName, cache, 'rebalanceThreadPoolSize', null, null, 1);
-        $generatorJava.property(res, varName, cache, 'rebalanceBatchSize', null, null, 524288);
-        $generatorJava.property(res, varName, cache, 'rebalanceBatchesPrefetchCount', null, null, 2);
-        $generatorJava.property(res, varName, cache, 'rebalanceOrder', null, null, 0);
-        $generatorJava.property(res, varName, cache, 'rebalanceDelay', null, null, 0);
-        $generatorJava.property(res, varName, cache, 'rebalanceTimeout', null, null, 10000);
-        $generatorJava.property(res, varName, cache, 'rebalanceThrottle', null, null, 0);
-    }
-
-    res.softEmptyLine();
-
-    if (cache.igfsAffinnityGroupSize) {
-        res.line(varName + '.setAffinityMapper(new ' + res.importClass('org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper') + '(' + cache.igfsAffinnityGroupSize + '));');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate cache server near cache group.
-$generatorJava.cacheServerNearCache = function(cache, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!varName)
-        varName = $generatorJava.nextVariableName('cache', cache);
-
-    if (cache.cacheMode === 'PARTITIONED' && cache.nearCacheEnabled) {
-        res.needEmptyLine = true;
-
-        if (cache.nearConfiguration) {
-            $generatorJava.declareVariable(res, 'nearCfg', 'org.apache.ignite.configuration.NearCacheConfiguration');
-
-            res.needEmptyLine = true;
-
-            if (cache.nearConfiguration.nearStartSize) {
-                $generatorJava.property(res, 'nearCfg', cache.nearConfiguration, 'nearStartSize', null, null, 375000);
-
-                res.needEmptyLine = true;
-            }
-
-            if (cache.nearConfiguration.nearEvictionPolicy && cache.nearConfiguration.nearEvictionPolicy.kind) {
-                $generatorJava.evictionPolicy(res, 'nearCfg', cache.nearConfiguration.nearEvictionPolicy, 'nearEvictionPolicy');
-
-                res.needEmptyLine = true;
-            }
-
-            res.line(varName + '.setNearConfiguration(nearCfg);');
-
-            res.needEmptyLine = true;
-        }
-    }
-
-    return res;
-};
-
-// Generate cache statistics group.
-$generatorJava.cacheStatistics = function(cache, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!varName)
-        varName = $generatorJava.nextVariableName('cache', cache);
-
-    $generatorJava.property(res, varName, cache, 'statisticsEnabled', null, null, false);
-    $generatorJava.property(res, varName, cache, 'managementEnabled', null, null, false);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate domain model query fields.
-$generatorJava.domainModelQueryFields = function(res, domain) {
-    const fields = domain.fields;
-
-    if (fields && fields.length > 0) {
-        $generatorJava.declareVariable(res, 'fields', 'java.util.LinkedHashMap', 'java.util.LinkedHashMap', 'java.lang.String', 'java.lang.String');
-
-        _.forEach(fields, function(field) {
-            res.line('fields.put("' + field.name + '", "' + $generatorCommon.JavaTypes.fullClassName(field.className) + '");');
-        });
-
-        res.needEmptyLine = true;
-
-        res.line('qryMeta.setFields(fields);');
-
-        res.needEmptyLine = true;
-    }
-};
-
-// Generate domain model query aliases.
-$generatorJava.domainModelQueryAliases = function(res, domain) {
-    const aliases = domain.aliases;
-
-    if (aliases && aliases.length > 0) {
-        $generatorJava.declareVariable(res, 'aliases', 'java.util.Map', 'java.util.HashMap', 'java.lang.String', 'java.lang.String');
-
-        _.forEach(aliases, function(alias) {
-            res.line('aliases.put("' + alias.field + '", "' + alias.alias + '");');
-        });
-
-        res.needEmptyLine = true;
-
-        res.line('qryMeta.setAliases(aliases);');
-
-        res.needEmptyLine = true;
-    }
-};
-
-// Generate domain model indexes.
-$generatorJava.domainModelQueryIndexes = function(res, domain) {
-    const indexes = domain.indexes;
-
-    if (indexes && indexes.length > 0) {
-        res.needEmptyLine = true;
-
-        $generatorJava.declareVariable(res, 'indexes', 'java.util.List', 'java.util.ArrayList', 'org.apache.ignite.cache.QueryIndex');
-
-        _.forEach(indexes, function(index) {
-            const fields = index.fields;
-
-            // One row generation for 1 field index.
-            if (fields && fields.length === 1) {
-                const field = index.fields[0];
-
-                res.line('indexes.add(new ' + res.importClass('org.apache.ignite.cache.QueryIndex') +
-                    '("' + field.name + '", ' +
-                    res.importClass('org.apache.ignite.cache.QueryIndexType') + '.' + index.indexType + ', ' +
-                    field.direction + ', "' + index.name + '"));');
-            }
-            else {
-                res.needEmptyLine = true;
-
-                $generatorJava.declareVariable(res, 'index', 'org.apache.ignite.cache.QueryIndex');
-
-                $generatorJava.property(res, 'index', index, 'name');
-                $generatorJava.enumProperty(res, 'index', index, 'indexType', 'org.apache.ignite.cache.QueryIndexType');
-
-                res.needEmptyLine = true;
-
-                if (fields && fields.length > 0) {
-                    $generatorJava.declareVariable(res, 'indFlds', 'java.util.LinkedHashMap', 'java.util.LinkedHashMap', 'String', 'Boolean');
-
-                    _.forEach(fields, function(field) {
-                        res.line('indFlds.put("' + field.name + '", ' + field.direction + ');');
-                    });
-
-                    res.needEmptyLine = true;
-
-                    res.line('index.setFields(indFlds);');
-
-                    res.needEmptyLine = true;
-                }
-
-                res.line('indexes.add(index);');
-            }
-        });
-
-        res.needEmptyLine = true;
-
-        res.line('qryMeta.setIndexes(indexes);');
-
-        res.needEmptyLine = true;
-    }
-};
-
-// Generate domain model db fields.
-$generatorJava.domainModelDatabaseFields = function(res, domain, fieldProperty) {
-    const dbFields = domain[fieldProperty];
-
-    if (dbFields && dbFields.length > 0) {
-        res.needEmptyLine = true;
-
-        res.importClass('java.sql.Types');
-
-        res.startBlock('jdbcType.' + $generatorCommon.toJavaName('set', fieldProperty) + '(');
-
-        const lastIx = dbFields.length - 1;
-
-        res.importClass('org.apache.ignite.cache.store.jdbc.JdbcTypeField');
-
-        _.forEach(dbFields, function(field, ix) {
-            res.line('new JdbcTypeField(' +
-                'Types.' + field.databaseFieldType + ', ' + '"' + field.databaseFieldName + '", ' +
-                res.importClass(field.javaFieldType) + '.class, ' + '"' + field.javaFieldName + '"' + ')' + (ix < lastIx ? ',' : ''));
-        });
-
-        res.endBlock(');');
-
-        res.needEmptyLine = true;
-    }
-};
-
-// Generate domain model general group.
-$generatorJava.domainModelGeneral = function(domain, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    switch ($generatorCommon.domainQueryMetadata(domain)) {
-        case 'Annotations':
-            if ($generatorCommon.isDefinedAndNotEmpty(domain.keyType) || $generatorCommon.isDefinedAndNotEmpty(domain.valueType)) {
-                const types = [];
-
-                if ($generatorCommon.isDefinedAndNotEmpty(domain.keyType))
-                    types.push($generatorJava.toJavaCode(res.importClass(domain.keyType), 'class'));
-                else
-                    types.push('???');
-
-                if ($generatorCommon.isDefinedAndNotEmpty(domain.valueType))
-                    types.push($generatorJava.toJavaCode(res.importClass(domain.valueType), 'class'));
-                else
-                    types.push('???');
-
-                if ($generatorCommon.isDefinedAndNotEmpty(types))
-                    $generatorJava.fxVarArgs(res, 'cache.setIndexedTypes', false, types);
-            }
-
-            break;
-
-        case 'Configuration':
-            $generatorJava.classNameProperty(res, 'jdbcTypes', domain, 'keyType');
-            $generatorJava.property(res, 'jdbcTypes', domain, 'valueType');
-
-            if ($generatorCommon.isDefinedAndNotEmpty(domain.fields)) {
-                res.needEmptyLine = true;
-
-                $generatorJava.classNameProperty(res, 'qryMeta', domain, 'keyType');
-                $generatorJava.property(res, 'qryMeta', domain, 'valueType');
-            }
-
-            break;
-
-        default:
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate domain model for query group.
-$generatorJava.domainModelQuery = function(domain, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.domainQueryMetadata(domain) === 'Configuration') {
-        $generatorJava.domainModelQueryFields(res, domain);
-        $generatorJava.domainModelQueryAliases(res, domain);
-        $generatorJava.domainModelQueryIndexes(res, domain);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate domain model for store group.
-$generatorJava.domainStore = function(domain, withTypes, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorJava.property(res, 'jdbcType', domain, 'databaseSchema');
-    $generatorJava.property(res, 'jdbcType', domain, 'databaseTable');
-
-    if (withTypes) {
-        $generatorJava.classNameProperty(res, 'jdbcType', domain, 'keyType');
-        $generatorJava.property(res, 'jdbcType', domain, 'valueType');
-    }
-
-    $generatorJava.domainModelDatabaseFields(res, domain, 'keyFields');
-    $generatorJava.domainModelDatabaseFields(res, domain, 'valueFields');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate domain model configs.
-$generatorJava.cacheDomains = function(domains, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    const domainConfigs = _.filter(domains, function(domain) {
-        return $generatorCommon.domainQueryMetadata(domain) === 'Configuration' &&
-            $generatorCommon.isDefinedAndNotEmpty(domain.fields);
-    });
-
-    // Generate domain model configs.
-    if ($generatorCommon.isDefinedAndNotEmpty(domainConfigs)) {
-        $generatorJava.declareVariable(res, 'queryEntities', 'java.util.Collection', 'java.util.ArrayList', 'org.apache.ignite.cache.QueryEntity');
-
-        _.forEach(domainConfigs, function(domain) {
-            if ($generatorCommon.isDefinedAndNotEmpty(domain.fields))
-                res.line('queryEntities.add(queryEntity' + $generatorJava.extractType(domain.valueType) + '());');
-        });
-
-        res.needEmptyLine = true;
-
-        res.line(varName + '.setQueryEntities(queryEntities);');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate cache configs.
-$generatorJava.cache = function(cache, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorJava.cacheGeneral(cache, varName, res);
-    $generatorJava.cacheMemory(cache, varName, res);
-    $generatorJava.cacheQuery(cache, cache.domains, varName, res);
-    $generatorJava.cacheStore(cache, cache.domains, varName, res);
-
-    const igfs = _.get(cache, 'nodeFilter.IGFS.instance');
-
-    $generatorJava.cacheNodeFilter(cache, igfs ? [igfs] : [], varName, res);
-    $generatorJava.cacheConcurrency(cache, varName, res);
-    $generatorJava.cacheRebalance(cache, varName, res);
-    $generatorJava.cacheServerNearCache(cache, varName, res);
-    $generatorJava.cacheStatistics(cache, varName, res);
-    $generatorJava.cacheDomains(cache.domains, varName, res);
-};
-
-// Generation of cache domain model in separate methods.
-$generatorJava.clusterDomains = function(caches, res) {
-    const domains = [];
-
-    const typeVarName = 'jdbcType';
-    const metaVarName = 'qryMeta';
-
-    _.forEach(caches, function(cache) {
-        _.forEach(cache.domains, function(domain) {
-            if (_.isNil(_.find(domains, function(m) {
-                return m === domain.valueType;
-            }))) {
-                $generatorJava.resetVariables(res);
-
-                const type = $generatorJava.extractType(domain.valueType);
-
-                if ($generatorCommon.isDefinedAndNotEmpty(domain.databaseTable)) {
-                    res.line('/**');
-                    res.line(' * Create JDBC type for ' + type + '.');
-                    res.line(' *');
-                    res.line(' * @param cacheName Cache name.');
-                    res.line(' * @return Configured JDBC type.');
-                    res.line(' */');
-                    res.startBlock('private static JdbcType jdbcType' + type + '(String cacheName) {');
-
-                    $generatorJava.declareVariable(res, typeVarName, 'org.apache.ignite.cache.store.jdbc.JdbcType');
-
-                    res.needEmptyLine = true;
-
-                    res.line(typeVarName + '.setCacheName(cacheName);');
-
-                    $generatorJava.domainStore(domain, true, res);
-
-                    res.needEmptyLine = true;
-
-                    res.line('return ' + typeVarName + ';');
-                    res.endBlock('}');
-
-                    res.needEmptyLine = true;
-                }
-
-                if ($generatorCommon.domainQueryMetadata(domain) === 'Configuration' &&
-                    $generatorCommon.isDefinedAndNotEmpty(domain.fields)) {
-                    res.line('/**');
-                    res.line(' * Create SQL Query descriptor for ' + type + '.');
-                    res.line(' *');
-                    res.line(' * @return Configured query entity.');
-                    res.line(' */');
-                    res.startBlock('private static QueryEntity queryEntity' + type + '() {');
-
-                    $generatorJava.declareVariable(res, metaVarName, 'org.apache.ignite.cache.QueryEntity');
-
-                    $generatorJava.classNameProperty(res, metaVarName, domain, 'keyType');
-                    $generatorJava.property(res, metaVarName, domain, 'valueType');
-
-                    res.needEmptyLine = true;
-
-                    $generatorJava.domainModelQuery(domain, res);
-
-                    res.emptyLineIfNeeded();
-                    res.line('return ' + metaVarName + ';');
-
-                    res.needEmptyLine = true;
-
-                    res.endBlock('}');
-                }
-
-                domains.push(domain.valueType);
-            }
-        });
-    });
-};
-
-/**
- * @param prefix Variable prefix.
- * @param obj Object to process.
- * @param names Known names to generate next unique name.
- */
-$generatorJava.nextVariableName = function(prefix, obj, names) {
-    let nextName = $generatorCommon.toJavaName(prefix, obj.name);
-
-    let ix = 0;
-
-    const checkNextName = (name) => name === nextName + (ix === 0 ? '' : '_' + ix);
-
-    while (_.find(names, (name) => checkNextName(name)))
-        ix++;
-
-    if (ix > 0)
-        nextName = nextName + '_' + ix;
-
-    return nextName;
-};
-
-// Generate cluster caches.
-$generatorJava.clusterCaches = function(caches, igfss, isSrvCfg, res) {
-    function clusterCache(cache, names) {
-        res.emptyLineIfNeeded();
-
-        const cacheName = $generatorJava.nextVariableName('cache', cache, names);
-
-        $generatorJava.resetVariables(res);
-
-        const hasDatasource = $generatorCommon.cacheHasDatasource(cache);
-
-        res.line('/**');
-        res.line(' * Create configuration for cache "' + cache.name + '".');
-        res.line(' *');
-        res.line(' * @return Configured cache.');
-
-        if (hasDatasource)
-            res.line(' * @throws Exception if failed to create cache configuration.');
-
-        res.line(' */');
-        res.startBlock('public static CacheConfiguration ' + cacheName + '()' + (hasDatasource ? ' throws Exception' : '') + ' {');
-
-        $generatorJava.declareVariable(res, cacheName, 'org.apache.ignite.configuration.CacheConfiguration');
-
-        $generatorJava.cache(cache, cacheName, res);
-
-        res.line('return ' + cacheName + ';');
-        res.endBlock('}');
-
-        names.push(cacheName);
-
-        res.needEmptyLine = true;
-    }
-
-    if (!res)
-        res = $generatorCommon.builder();
-
-    const names = [];
-
-    if ($generatorCommon.isDefinedAndNotEmpty(caches)) {
-        res.emptyLineIfNeeded();
-
-        _.forEach(caches, function(cache) {
-            clusterCache(cache, names);
-        });
-
-        res.needEmptyLine = true;
-    }
-
-    if (isSrvCfg && $generatorCommon.isDefinedAndNotEmpty(igfss)) {
-        res.emptyLineIfNeeded();
-
-        _.forEach(igfss, function(igfs) {
-            clusterCache($generatorCommon.igfsDataCache(igfs), names);
-            clusterCache($generatorCommon.igfsMetaCache(igfs), names);
-        });
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate cluster caches.
-$generatorJava.clusterCacheUse = function(caches, igfss, res) {
-    function clusterCacheInvoke(cache, names) {
-        names.push($generatorJava.nextVariableName('cache', cache, names));
-    }
-
-    if (!res)
-        res = $generatorCommon.builder();
-
-    const cacheNames = [];
-
-    _.forEach(caches, function(cache) {
-        clusterCacheInvoke(cache, cacheNames);
-    });
-
-    const igfsNames = [];
-
-    _.forEach(igfss, function(igfs) {
-        clusterCacheInvoke($generatorCommon.igfsDataCache(igfs), igfsNames);
-        clusterCacheInvoke($generatorCommon.igfsMetaCache(igfs), igfsNames);
-    });
-
-    const allCacheNames = cacheNames.concat(igfsNames);
-
-    if (allCacheNames.length) {
-        res.line('cfg.setCacheConfiguration(' + allCacheNames.join('(), ') + '());');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Get class name from fully specified class path.
-$generatorJava.extractType = function(fullType) {
-    return fullType.substring(fullType.lastIndexOf('.') + 1);
-};
-
-/**
- * Generate java class code.
- *
- * @param domain Domain model object.
- * @param key If 'true' then key class should be generated.
- * @param pkg Package name.
- * @param useConstructor If 'true' then empty and full constructors should be generat

<TRUNCATED>

[29/50] [abbrv] ignite git commit: Web console beta-7.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.platform.provider.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.platform.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.platform.provider.js
deleted file mode 100644
index 582426e..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.platform.provider.js
+++ /dev/null
@@ -1,49 +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.
- */
-
-import _ from 'lodash';
-
-const enumValueMapper = (val) => _.capitalize(val);
-
-const DFLT_CLUSTER = {
-    atomics: {
-        cacheMode: {
-            clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheMode',
-            mapper: enumValueMapper
-        }
-    },
-    transactionConfiguration: {
-        defaultTxConcurrency: {
-            clsName: 'Apache.Ignite.Core.Transactions.TransactionConcurrency',
-            mapper: enumValueMapper
-        },
-        defaultTxIsolation: {
-            clsName: 'Apache.Ignite.Core.Transactions.TransactionIsolation',
-            mapper: enumValueMapper
-        }
-    }
-};
-
-export default function() {
-    this.append = (dflts) => {
-        _.merge(DFLT_CLUSTER, dflts);
-    };
-
-    this.$get = ['igniteClusterDefaults', (clusterDefaults) => {
-        return _.merge({}, clusterDefaults, DFLT_CLUSTER);
-    }];
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js
deleted file mode 100644
index 726581d..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js
+++ /dev/null
@@ -1,293 +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.
- */
-
-const DFLT_CLUSTER = {
-    localHost: '0.0.0.0',
-    discovery: {
-        localPort: 47500,
-        localPortRange: 100,
-        socketTimeout: 5000,
-        ackTimeout: 5000,
-        maxAckTimeout: 600000,
-        networkTimeout: 5000,
-        joinTimeout: 0,
-        threadPriority: 10,
-        heartbeatFrequency: 2000,
-        maxMissedHeartbeats: 1,
-        maxMissedClientHeartbeats: 5,
-        topHistorySize: 1000,
-        reconnectCount: 10,
-        statisticsPrintFrequency: 0,
-        ipFinderCleanFrequency: 60000,
-        forceServerMode: false,
-        clientReconnectDisabled: false,
-        Multicast: {
-            multicastGroup: '228.1.2.4',
-            multicastPort: 47400,
-            responseWaitTime: 500,
-            addressRequestAttempts: 2,
-            localAddress: '0.0.0.0'
-        },
-        Jdbc: {
-            initSchema: false
-        },
-        SharedFs: {
-            path: 'disco/tcp'
-        },
-        ZooKeeper: {
-            basePath: '/services',
-            serviceName: 'ignite',
-            allowDuplicateRegistrations: false,
-            ExponentialBackoff: {
-                baseSleepTimeMs: 1000,
-                maxRetries: 10
-            },
-            BoundedExponentialBackoffRetry: {
-                baseSleepTimeMs: 1000,
-                maxSleepTimeMs: 2147483647,
-                maxRetries: 10
-            },
-            UntilElapsed: {
-                maxElapsedTimeMs: 60000,
-                sleepMsBetweenRetries: 1000
-            },
-            RetryNTimes: {
-                n: 10,
-                sleepMsBetweenRetries: 1000
-            },
-            OneTime: {
-                sleepMsBetweenRetry: 1000
-            },
-            Forever: {
-                retryIntervalMs: 1000
-            }
-        }
-    },
-    atomics: {
-        atomicSequenceReserveSize: 1000,
-        backups: 0,
-        cacheMode: {
-            clsName: 'org.apache.ignite.cache.CacheMode',
-            value: 'PARTITIONED'
-        }
-    },
-    binary: {
-        compactFooter: true,
-        typeConfigurations: {
-            enum: false
-        }
-    },
-    collision: {
-        kind: null,
-        JobStealing: {
-            activeJobsThreshold: 95,
-            waitJobsThreshold: 0,
-            messageExpireTime: 1000,
-            maximumStealingAttempts: 5,
-            stealingEnabled: true,
-            stealingAttributes: {
-                keyClsName: 'java.lang.String',
-                valClsName: 'java.io.Serializable',
-                items: []
-            }
-        },
-        PriorityQueue: {
-            priorityAttributeKey: 'grid.task.priority',
-            jobPriorityAttributeKey: 'grid.job.priority',
-            defaultPriority: 0,
-            starvationIncrement: 1,
-            starvationPreventionEnabled: true
-        }
-    },
-    communication: {
-        localPort: 47100,
-        localPortRange: 100,
-        sharedMemoryPort: 48100,
-        directBuffer: false,
-        directSendBuffer: false,
-        idleConnectionTimeout: 30000,
-        connectTimeout: 5000,
-        maxConnectTimeout: 600000,
-        reconnectCount: 10,
-        socketSendBuffer: 32768,
-        socketReceiveBuffer: 32768,
-        messageQueueLimit: 1024,
-        tcpNoDelay: true,
-        ackSendThreshold: 16,
-        unacknowledgedMessagesBufferSize: 0,
-        socketWriteTimeout: 2000
-    },
-    networkTimeout: 5000,
-    networkSendRetryDelay: 1000,
-    networkSendRetryCount: 3,
-    discoveryStartupDelay: 60000,
-    connector: {
-        port: 11211,
-        portRange: 100,
-        idleTimeout: 7000,
-        idleQueryCursorTimeout: 600000,
-        idleQueryCursorCheckFrequency: 60000,
-        receiveBufferSize: 32768,
-        sendBufferSize: 32768,
-        sendQueueLimit: 0,
-        directBuffer: false,
-        noDelay: true,
-        sslEnabled: false,
-        sslClientAuth: false
-    },
-    deploymentMode: {
-        clsName: 'org.apache.ignite.configuration.DeploymentMode',
-        value: 'SHARED'
-    },
-    peerClassLoadingEnabled: false,
-    peerClassLoadingMissedResourcesCacheSize: 100,
-    peerClassLoadingThreadPoolSize: 2,
-    failoverSpi: {
-        JobStealing: {
-            maximumFailoverAttempts: 5
-        },
-        Always: {
-            maximumFailoverAttempts: 5
-        }
-    },
-    logger: {
-        Log4j: {
-            level: {
-                clsName: 'org.apache.log4j.Level'
-            }
-        },
-        Log4j2: {
-            level: {
-                clsName: 'org.apache.logging.log4j.Level'
-            }
-        }
-    },
-    marshalLocalJobs: false,
-    marshallerCacheKeepAliveTime: 10000,
-    metricsHistorySize: 10000,
-    metricsLogFrequency: 60000,
-    metricsUpdateFrequency: 2000,
-    clockSyncSamples: 8,
-    clockSyncFrequency: 120000,
-    timeServerPortBase: 31100,
-    timeServerPortRange: 100,
-    transactionConfiguration: {
-        defaultTxConcurrency: {
-            clsName: 'org.apache.ignite.transactions.TransactionConcurrency',
-            value: 'PESSIMISTIC'
-        },
-        defaultTxIsolation: {
-            clsName: 'org.apache.ignite.transactions.TransactionIsolation',
-            value: 'REPEATABLE_READ'
-        },
-        defaultTxTimeout: 0,
-        pessimisticTxLogLinger: 10000
-    },
-    attributes: {
-        keyClsName: 'java.lang.String',
-        valClsName: 'java.lang.String',
-        items: []
-    },
-    odbcConfiguration: {
-        endpointAddress: '0.0.0.0:10800..10810',
-        maxOpenCursors: 128
-    },
-    eventStorage: {
-        Memory: {
-            expireCount: 10000
-        }
-    },
-    checkpointSpi: {
-        S3: {
-            bucketNameSuffix: 'default-bucket',
-            clientConfiguration: {
-                protocol: {
-                    clsName: 'com.amazonaws.Protocol',
-                    value: 'HTTPS'
-                },
-                maxConnections: 50,
-                retryPolicy: {
-                    retryCondition: {
-                        clsName: 'com.amazonaws.retry.PredefinedRetryPolicies'
-                    },
-                    backoffStrategy: {
-                        clsName: 'com.amazonaws.retry.PredefinedRetryPolicies'
-                    },
-                    maxErrorRetry: {
-                        clsName: 'com.amazonaws.retry.PredefinedRetryPolicies'
-                    }
-                },
-                maxErrorRetry: -1,
-                socketTimeout: 50000,
-                connectionTimeout: 50000,
-                requestTimeout: 0,
-                socketSendBufferSizeHints: 0,
-                connectionTTL: -1,
-                connectionMaxIdleMillis: 60000,
-                responseMetadataCacheSize: 50,
-                useReaper: true,
-                useGzip: false,
-                preemptiveBasicProxyAuth: false,
-                useTcpKeepAlive: false
-            }
-        },
-        JDBC: {
-            checkpointTableName: 'CHECKPOINTS',
-            keyFieldName: 'NAME',
-            keyFieldType: 'VARCHAR',
-            valueFieldName: 'VALUE',
-            valueFieldType: 'BLOB',
-            expireDateFieldName: 'EXPIRE_DATE',
-            expireDateFieldType: 'DATETIME',
-            numberOfRetries: 2
-        }
-    },
-    loadBalancingSpi: {
-        RoundRobin: {
-            perTask: false
-        },
-        Adaptive: {
-            loadProbe: {
-                Job: {
-                    useAverage: true
-                },
-                CPU: {
-                    useAverage: true,
-                    useProcessors: true,
-                    processorCoefficient: 1
-                },
-                ProcessingTime: {
-                    useAverage: true
-                }
-            }
-        },
-        WeightedRandom: {
-            nodeWeight: 10,
-            useWeights: false
-        }
-    }
-};
-
-export default function() {
-    this.append = (dflts) => {
-        _.merge(DFLT_CLUSTER, dflts);
-    };
-
-    this.$get = [() => {
-        return DFLT_CLUSTER;
-    }];
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/igfs.provider.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/igfs.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/igfs.provider.js
deleted file mode 100644
index c556336..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/igfs.provider.js
+++ /dev/null
@@ -1,68 +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.
- */
-
-const DFLT_IGFS = {
-    defaultMode: {
-        clsName: 'org.apache.ignite.igfs.IgfsMode',
-        value: 'DUAL_ASYNC'
-    },
-    secondaryFileSystem: {
-
-    },
-    ipcEndpointConfiguration: {
-        type: {
-            clsName: 'org.apache.ignite.igfs.IgfsIpcEndpointType'
-        },
-        host: '127.0.0.1',
-        port: 10500,
-        memorySize: 262144,
-        tokenDirectoryPath: 'ipc/shmem'
-    },
-    fragmentizerConcurrentFiles: 0,
-    fragmentizerThrottlingBlockLength: 16777216,
-    fragmentizerThrottlingDelay: 200,
-    dualModeMaxPendingPutsSize: 0,
-    dualModePutExecutorServiceShutdown: false,
-    blockSize: 65536,
-    streamBufferSize: 65536,
-    maxSpaceSize: 0,
-    maximumTaskRangeLength: 0,
-    managementPort: 11400,
-    perNodeBatchSize: 100,
-    perNodeParallelBatchCount: 8,
-    prefetchBlocks: 0,
-    sequentialReadsBeforePrefetch: 0,
-    trashPurgeTimeout: 1000,
-    colocateMetadata: true,
-    relaxedConsistency: true,
-    pathModes: {
-        keyClsName: 'java.lang.String',
-        keyField: 'path',
-        valClsName: 'org.apache.ignite.igfs.IgfsMode',
-        valField: 'mode'
-    }
-};
-
-export default function() {
-    this.append = (dflts) => {
-        _.merge(DFLT_IGFS, dflts);
-    };
-
-    this.$get = [() => {
-        return DFLT_IGFS;
-    }];
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/generator-common.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/generator-common.js b/modules/web-console/frontend/app/modules/configuration/generator/generator-common.js
deleted file mode 100644
index d502c8a..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/generator-common.js
+++ /dev/null
@@ -1,625 +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.
- */
-
-// Entry point for common functions for code generation.
-const $generatorCommon = {};
-
-// Add leading zero.
-$generatorCommon.addLeadingZero = function(numberStr, minSize) {
-    if (typeof (numberStr) !== 'string')
-        numberStr = String(numberStr);
-
-    while (numberStr.length < minSize)
-        numberStr = '0' + numberStr;
-
-    return numberStr;
-};
-
-// Format date to string.
-$generatorCommon.formatDate = function(date) {
-    const dd = $generatorCommon.addLeadingZero(date.getDate(), 2);
-    const mm = $generatorCommon.addLeadingZero(date.getMonth() + 1, 2);
-
-    const yyyy = date.getFullYear();
-
-    return mm + '/' + dd + '/' + yyyy + ' ' + $generatorCommon.addLeadingZero(date.getHours(), 2) + ':' + $generatorCommon.addLeadingZero(date.getMinutes(), 2);
-};
-
-/**
- * Generate title comment for XML, Java, ... files.
- *
- * @param sbj {string} What is generated.
- * @returns {string} Text to add as title comment in generated java class.
- */
-$generatorCommon.mainComment = function mainComment(sbj) {
-    return 'This ' + sbj + ' was generated by Ignite Web Console (' + $generatorCommon.formatDate(new Date()) + ')';
-};
-
-// Create result holder with service functions and properties for XML and java code generation.
-$generatorCommon.builder = function(deep) {
-    if (_.isNil($generatorCommon.JavaTypes))
-        $generatorCommon.JavaTypes = angular.element(document.getElementById('app')).injector().get('JavaTypes');
-
-    const res = [];
-
-    res.deep = deep || 0;
-    res.needEmptyLine = false;
-    res.lineStart = true;
-    res.datasources = [];
-    res.imports = {};
-    res.staticImports = {};
-    res.vars = {};
-
-    res.safeDeep = 0;
-    res.safeNeedEmptyLine = false;
-    res.safeImports = {};
-    res.safeDatasources = [];
-    res.safePoint = -1;
-
-    res.mergeProps = function(fromRes) {
-        if ($generatorCommon.isDefinedAndNotEmpty(fromRes)) {
-            res.datasources = fromRes.datasources;
-
-            angular.extend(res.imports, fromRes.imports);
-            angular.extend(res.staticImports, fromRes.staticImports);
-            angular.extend(res.vars, fromRes.vars);
-        }
-    };
-
-    res.mergeLines = function(fromRes) {
-        if ($generatorCommon.isDefinedAndNotEmpty(fromRes)) {
-            if (res.needEmptyLine)
-                res.push('');
-
-            _.forEach(fromRes, function(line) {
-                res.append(line);
-            });
-        }
-    };
-
-    res.startSafeBlock = function() {
-        res.safeDeep = this.deep;
-        this.safeNeedEmptyLine = this.needEmptyLine;
-        this.safeImports = _.cloneDeep(this.imports);
-        this.safeStaticImports = _.cloneDeep(this.staticImports);
-        this.safeDatasources = this.datasources.slice();
-        this.safePoint = this.length;
-    };
-
-    res.rollbackSafeBlock = function() {
-        if (this.safePoint >= 0) {
-            this.splice(this.safePoint, this.length - this.safePoint);
-
-            this.deep = res.safeDeep;
-            this.needEmptyLine = this.safeNeedEmptyLine;
-            this.datasources = this.safeDatasources;
-            this.imports = this.safeImports;
-            this.staticImports = this.safeStaticImports;
-            this.safePoint = -1;
-        }
-    };
-
-    res.asString = function() {
-        return this.join('\n');
-    };
-
-    res.append = function(s) {
-        this.push((this.lineStart ? _.repeat('    ', this.deep) : '') + s);
-
-        return this;
-    };
-
-    res.line = function(s) {
-        if (s) {
-            if (res.needEmptyLine)
-                res.push('');
-
-            res.append(s);
-        }
-
-        res.needEmptyLine = false;
-
-        res.lineStart = true;
-
-        return res;
-    };
-
-    res.startBlock = function(s) {
-        if (s) {
-            if (this.needEmptyLine)
-                this.push('');
-
-            this.append(s);
-        }
-
-        this.needEmptyLine = false;
-
-        this.lineStart = true;
-
-        this.deep++;
-
-        return this;
-    };
-
-    res.endBlock = function(s) {
-        this.deep--;
-
-        if (s)
-            this.append(s);
-
-        this.lineStart = true;
-
-        return this;
-    };
-
-    res.softEmptyLine = function() {
-        this.needEmptyLine = this.length > 0;
-    };
-
-    res.emptyLineIfNeeded = function() {
-        if (this.needEmptyLine) {
-            this.push('');
-            this.lineStart = true;
-
-            this.needEmptyLine = false;
-        }
-    };
-
-    /**
-     * Add class to imports.
-     *
-     * @param clsName Full class name.
-     * @returns {String} Short class name or full class name in case of names conflict.
-     */
-    res.importClass = function(clsName) {
-        if ($generatorCommon.JavaTypes.isJavaPrimitive(clsName))
-            return clsName;
-
-        const fullClassName = $generatorCommon.JavaTypes.fullClassName(clsName);
-
-        const dotIdx = fullClassName.lastIndexOf('.');
-
-        const shortName = dotIdx > 0 ? fullClassName.substr(dotIdx + 1) : fullClassName;
-
-        if (this.imports[shortName]) {
-            if (this.imports[shortName] !== fullClassName)
-                return fullClassName; // Short class names conflict. Return full name.
-        }
-        else
-            this.imports[shortName] = fullClassName;
-
-        return shortName;
-    };
-
-    /**
-     * Add class to imports.
-     *
-     * @param member Static member.
-     * @returns {String} Short class name or full class name in case of names conflict.
-     */
-    res.importStatic = function(member) {
-        const dotIdx = member.lastIndexOf('.');
-
-        const shortName = dotIdx > 0 ? member.substr(dotIdx + 1) : member;
-
-        if (this.staticImports[shortName]) {
-            if (this.staticImports[shortName] !== member)
-                return member; // Short class names conflict. Return full name.
-        }
-        else
-            this.staticImports[shortName] = member;
-
-        return shortName;
-    };
-
-    /**
-     * @returns String with "java imports" section.
-     */
-    res.generateImports = function() {
-        const genImports = [];
-
-        for (const clsName in this.imports) {
-            if (this.imports.hasOwnProperty(clsName) && this.imports[clsName].lastIndexOf('java.lang.', 0) !== 0)
-                genImports.push('import ' + this.imports[clsName] + ';');
-        }
-
-        genImports.sort();
-
-        return genImports.join('\n');
-    };
-
-    /**
-     * @returns String with "java imports" section.
-     */
-    res.generateStaticImports = function() {
-        const statImports = [];
-
-        for (const clsName in this.staticImports) {
-            if (this.staticImports.hasOwnProperty(clsName) && this.staticImports[clsName].lastIndexOf('java.lang.', 0) !== 0)
-                statImports.push('import static ' + this.staticImports[clsName] + ';');
-        }
-
-        statImports.sort();
-
-        return statImports.join('\n');
-    };
-
-    return res;
-};
-
-// Eviction policies code generation descriptors.
-$generatorCommon.EVICTION_POLICIES = {
-    LRU: {
-        className: 'org.apache.ignite.cache.eviction.lru.LruEvictionPolicy',
-        fields: {batchSize: {dflt: 1}, maxMemorySize: null, maxSize: {dflt: 100000}}
-    },
-    FIFO: {
-        className: 'org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy',
-        fields: {batchSize: {dflt: 1}, maxMemorySize: null, maxSize: {dflt: 100000}}
-    },
-    SORTED: {
-        className: 'org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicy',
-        fields: {batchSize: {dflt: 1}, maxMemorySize: null, maxSize: {dflt: 100000}}
-    }
-};
-
-// Marshaller code generation descriptors.
-$generatorCommon.MARSHALLERS = {
-    OptimizedMarshaller: {
-        className: 'org.apache.ignite.marshaller.optimized.OptimizedMarshaller',
-        fields: {poolSize: null, requireSerializable: null }
-    },
-    JdkMarshaller: {
-        className: 'org.apache.ignite.marshaller.jdk.JdkMarshaller',
-        fields: {}
-    }
-};
-
-// Pairs of supported databases and their JDBC dialects.
-$generatorCommon.JDBC_DIALECTS = {
-    Generic: 'org.apache.ignite.cache.store.jdbc.dialect.BasicJdbcDialect',
-    Oracle: 'org.apache.ignite.cache.store.jdbc.dialect.OracleDialect',
-    DB2: 'org.apache.ignite.cache.store.jdbc.dialect.DB2Dialect',
-    SQLServer: 'org.apache.ignite.cache.store.jdbc.dialect.SQLServerDialect',
-    MySQL: 'org.apache.ignite.cache.store.jdbc.dialect.MySQLDialect',
-    PostgreSQL: 'org.apache.ignite.cache.store.jdbc.dialect.BasicJdbcDialect',
-    H2: 'org.apache.ignite.cache.store.jdbc.dialect.H2Dialect'
-};
-
-// Return JDBC dialect full class name for specified database.
-$generatorCommon.jdbcDialectClassName = function(db) {
-    const dialectClsName = $generatorCommon.JDBC_DIALECTS[db];
-
-    return dialectClsName ? dialectClsName : 'Unknown database: ' + db;
-};
-
-// Generate default data cache for specified igfs instance.
-$generatorCommon.igfsDataCache = function(igfs) {
-    return {
-        name: igfs.name + '-data',
-        cacheMode: 'PARTITIONED',
-        atomicityMode: 'TRANSACTIONAL',
-        writeSynchronizationMode: 'FULL_SYNC',
-        backups: 0,
-        igfsAffinnityGroupSize: igfs.affinnityGroupSize || 512
-    };
-};
-
-// Generate default meta cache for specified igfs instance.
-$generatorCommon.igfsMetaCache = function(igfs) {
-    return {
-        name: igfs.name + '-meta',
-        cacheMode: 'REPLICATED',
-        atomicityMode: 'TRANSACTIONAL',
-        writeSynchronizationMode: 'FULL_SYNC'
-    };
-};
-
-// Pairs of supported databases and their data sources.
-$generatorCommon.DATA_SOURCES = {
-    Generic: 'com.mchange.v2.c3p0.ComboPooledDataSource',
-    Oracle: 'oracle.jdbc.pool.OracleDataSource',
-    DB2: 'com.ibm.db2.jcc.DB2DataSource',
-    SQLServer: 'com.microsoft.sqlserver.jdbc.SQLServerDataSource',
-    MySQL: 'com.mysql.jdbc.jdbc2.optional.MysqlDataSource',
-    PostgreSQL: 'org.postgresql.ds.PGPoolingDataSource',
-    H2: 'org.h2.jdbcx.JdbcDataSource'
-};
-
-// Return data source full class name for specified database.
-$generatorCommon.dataSourceClassName = function(db) {
-    const dsClsName = $generatorCommon.DATA_SOURCES[db];
-
-    return dsClsName ? dsClsName : 'Unknown database: ' + db;
-};
-
-// Store factories code generation descriptors.
-$generatorCommon.STORE_FACTORIES = {
-    CacheJdbcPojoStoreFactory: {
-        className: 'org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory',
-        suffix: 'JdbcPojo',
-        fields: {
-            configuration: {type: 'bean'}
-        }
-    },
-    CacheJdbcBlobStoreFactory: {
-        className: 'org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactory',
-        suffix: 'JdbcBlob',
-        fields: {
-            initSchema: null,
-            createTableQuery: null,
-            loadQuery: null,
-            insertQuery: null,
-            updateQuery: null,
-            deleteQuery: null
-        }
-    },
-    CacheHibernateBlobStoreFactory: {
-        className: 'org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreFactory',
-        suffix: 'Hibernate',
-        fields: {hibernateProperties: {type: 'propertiesAsList', propVarName: 'props'}}
-    }
-};
-
-// Swap space SPI code generation descriptor.
-$generatorCommon.SWAP_SPACE_SPI = {
-    className: 'org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi',
-    fields: {
-        baseDirectory: {type: 'path'},
-        readStripesNumber: null,
-        maximumSparsity: {type: 'float'},
-        maxWriteQueueSize: null,
-        writeBufferSize: null
-    }
-};
-
-// Transaction configuration code generation descriptor.
-$generatorCommon.TRANSACTION_CONFIGURATION = {
-    className: 'org.apache.ignite.configuration.TransactionConfiguration',
-    fields: {
-        defaultTxConcurrency: {type: 'enum', enumClass: 'org.apache.ignite.transactions.TransactionConcurrency', dflt: 'PESSIMISTIC'},
-        defaultTxIsolation: {type: 'enum', enumClass: 'org.apache.ignite.transactions.TransactionIsolation', dflt: 'REPEATABLE_READ'},
-        defaultTxTimeout: {dflt: 0},
-        pessimisticTxLogLinger: {dflt: 10000},
-        pessimisticTxLogSize: null,
-        txSerializableEnabled: null,
-        txManagerFactory: {type: 'bean'}
-    }
-};
-
-// SSL configuration code generation descriptor.
-$generatorCommon.SSL_CONFIGURATION_TRUST_FILE_FACTORY = {
-    className: 'org.apache.ignite.ssl.SslContextFactory',
-    fields: {
-        keyAlgorithm: null,
-        keyStoreFilePath: {type: 'path'},
-        keyStorePassword: {type: 'raw'},
-        keyStoreType: null,
-        protocol: null,
-        trustStoreFilePath: {type: 'path'},
-        trustStorePassword: {type: 'raw'},
-        trustStoreType: null
-    }
-};
-
-// SSL configuration code generation descriptor.
-$generatorCommon.SSL_CONFIGURATION_TRUST_MANAGER_FACTORY = {
-    className: 'org.apache.ignite.ssl.SslContextFactory',
-    fields: {
-        keyAlgorithm: null,
-        keyStoreFilePath: {type: 'path'},
-        keyStorePassword: {type: 'raw'},
-        keyStoreType: null,
-        protocol: null,
-        trustManagers: {type: 'array'}
-    }
-};
-
-// Communication configuration code generation descriptor.
-$generatorCommon.CONNECTOR_CONFIGURATION = {
-    className: 'org.apache.ignite.configuration.ConnectorConfiguration',
-    fields: {
-        jettyPath: null,
-        host: null,
-        port: {dflt: 11211},
-        portRange: {dflt: 100},
-        idleTimeout: {dflt: 7000},
-        idleQueryCursorTimeout: {dflt: 600000},
-        idleQueryCursorCheckFrequency: {dflt: 60000},
-        receiveBufferSize: {dflt: 32768},
-        sendBufferSize: {dflt: 32768},
-        sendQueueLimit: {dflt: 0},
-        directBuffer: {dflt: false},
-        noDelay: {dflt: true},
-        selectorCount: null,
-        threadPoolSize: null,
-        messageInterceptor: {type: 'bean'},
-        secretKey: null,
-        sslEnabled: {dflt: false}
-    }
-};
-
-// Communication configuration code generation descriptor.
-$generatorCommon.COMMUNICATION_CONFIGURATION = {
-    className: 'org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi',
-    fields: {
-        listener: {type: 'bean'},
-        localAddress: null,
-        localPort: {dflt: 47100},
-        localPortRange: {dflt: 100},
-        sharedMemoryPort: {dflt: 48100},
-        directBuffer: {dflt: false},
-        directSendBuffer: {dflt: false},
-        idleConnectionTimeout: {dflt: 30000},
-        connectTimeout: {dflt: 5000},
-        maxConnectTimeout: {dflt: 600000},
-        reconnectCount: {dflt: 10},
-        socketSendBuffer: {dflt: 32768},
-        socketReceiveBuffer: {dflt: 32768},
-        messageQueueLimit: {dflt: 1024},
-        slowClientQueueLimit: null,
-        tcpNoDelay: {dflt: true},
-        ackSendThreshold: {dflt: 16},
-        unacknowledgedMessagesBufferSize: {dflt: 0},
-        socketWriteTimeout: {dflt: 2000},
-        selectorsCount: null,
-        addressResolver: {type: 'bean'}
-    }
-};
-
-// Communication configuration code generation descriptor.
-$generatorCommon.IGFS_IPC_CONFIGURATION = {
-    className: 'org.apache.ignite.igfs.IgfsIpcEndpointConfiguration',
-    fields: {
-        type: {type: 'enum', enumClass: 'org.apache.ignite.igfs.IgfsIpcEndpointType'},
-        host: {dflt: '127.0.0.1'},
-        port: {dflt: 10500},
-        memorySize: {dflt: 262144},
-        tokenDirectoryPath: {dflt: 'ipc/shmem'},
-        threadCount: null
-    }
-};
-
-$generatorCommon.ODBC_CONFIGURATION = {
-    className: 'org.apache.ignite.configuration.OdbcConfiguration',
-    fields: {
-        endpointAddress: {dflt: '0.0.0.0:10800..10810'},
-        maxOpenCursors: {dflt: 128}
-    }
-};
-
-// Check that cache has datasource.
-$generatorCommon.cacheHasDatasource = function(cache) {
-    if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) {
-        const storeFactory = cache.cacheStoreFactory[cache.cacheStoreFactory.kind];
-
-        return !!(storeFactory && (storeFactory.connectVia ? (storeFactory.connectVia === 'DataSource' ? storeFactory.dialect : false) : storeFactory.dialect)); // eslint-disable-line no-nested-ternary
-    }
-
-    return false;
-};
-
-$generatorCommon.secretPropertiesNeeded = function(cluster) {
-    return !_.isNil(_.find(cluster.caches, $generatorCommon.cacheHasDatasource)) || cluster.sslEnabled;
-};
-
-// Check that binary is configured.
-$generatorCommon.binaryIsDefined = function(binary) {
-    return binary && ($generatorCommon.isDefinedAndNotEmpty(binary.idMapper) || $generatorCommon.isDefinedAndNotEmpty(binary.nameMapper) ||
-        $generatorCommon.isDefinedAndNotEmpty(binary.serializer) || $generatorCommon.isDefinedAndNotEmpty(binary.typeConfigurations) ||
-        (!_.isNil(binary.compactFooter) && !binary.compactFooter));
-};
-
-// Extract domain model metadata location.
-$generatorCommon.domainQueryMetadata = function(domain) {
-    return domain.queryMetadata ? domain.queryMetadata : 'Configuration';
-};
-
-/**
- * @param {Object} obj Object to check.
- * @param {Array<String>} props Array of properties names.
- * @returns {boolean} 'true' if
- */
-$generatorCommon.hasAtLeastOneProperty = function(obj, props) {
-    return obj && props && _.findIndex(props, (prop) => !_.isNil(obj[prop])) >= 0;
-};
-
-/**
- * Convert some name to valid java name.
- *
- * @param prefix To append to java name.
- * @param name to convert.
- * @returns {string} Valid java name.
- */
-$generatorCommon.toJavaName = function(prefix, name) {
-    const javaName = name ? name.replace(/[^A-Za-z_0-9]+/g, '_') : 'dflt';
-
-    return prefix + javaName.charAt(0).toLocaleUpperCase() + javaName.slice(1);
-};
-
-/**
- * @param v Value to check.
- * @returns {boolean} 'true' if value defined and not empty string.
- */
-$generatorCommon.isDefinedAndNotEmpty = function(v) {
-    let defined = !_.isNil(v);
-
-    if (defined && (_.isString(v) || _.isArray(v)))
-        defined = v.length > 0;
-
-    return defined;
-};
-
-/**
- * @param {Object} obj Object to check.
- * @param {Array<String>} props Properties names.
- * @returns {boolean} 'true' if object contains at least one from specified properties.
- */
-$generatorCommon.hasProperty = function(obj, props) {
-    for (const propName in props) {
-        if (props.hasOwnProperty(propName)) {
-            if (obj[propName])
-                return true;
-        }
-    }
-
-    return false;
-};
-
-/**
- * Get class for selected implementation of Failover SPI.
- *
- * @param spi Failover SPI configuration.
- * @returns {*} Class for selected implementation of Failover SPI.
- */
-$generatorCommon.failoverSpiClass = function(spi) {
-    switch (spi.kind) {
-        case 'JobStealing': return 'org.apache.ignite.spi.failover.jobstealing.JobStealingFailoverSpi';
-        case 'Never': return 'org.apache.ignite.spi.failover.never.NeverFailoverSpi';
-        case 'Always': return 'org.apache.ignite.spi.failover.always.AlwaysFailoverSpi';
-        case 'Custom': return _.get(spi, 'Custom.class');
-        default: return 'Unknown';
-    }
-};
-
-$generatorCommon.loggerConfigured = function(logger) {
-    if (logger && logger.kind) {
-        const log = logger[logger.kind];
-
-        switch (logger.kind) {
-            case 'Log4j2': return log && $generatorCommon.isDefinedAndNotEmpty(log.path);
-
-            case 'Log4j':
-                if (!log || !log.mode)
-                    return false;
-
-                if (log.mode === 'Path')
-                    return $generatorCommon.isDefinedAndNotEmpty(log.path);
-
-                return true;
-
-            case 'Custom': return log && $generatorCommon.isDefinedAndNotEmpty(log.class);
-
-            default:
-                return true;
-        }
-    }
-
-    return false;
-};
-
-export default $generatorCommon;


[08/50] [abbrv] ignite git commit: IGNITE-4359 .NET: Support DateTime properties in LINQ

Posted by yz...@apache.org.
IGNITE-4359 .NET: Support DateTime properties in LINQ


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 3bae858986f78dbb288cf39d970b3f163c3da247
Parents: 126ab60
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Dec 20 15:32:39 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Dec 20 15:32:39 2016 +0300

----------------------------------------------------------------------
 .../Cache/Query/CacheLinqTest.cs                | 24 ++++++++++----
 .../Impl/CacheQueryExpressionVisitor.cs         | 12 ++-----
 .../Apache.Ignite.Linq/Impl/MethodVisitor.cs    | 34 ++++++++++++++++++--
 3 files changed, 52 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3bae8589/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
index 1ac7fa7..46979ef 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
@@ -63,7 +63,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         private bool _runDbConsole;
 
         /** */
-        private static readonly DateTime StartDateTime = new DateTime(2000, 1, 1, 0, 0, 0, DateTimeKind.Utc);
+        private static readonly DateTime StartDateTime = new DateTime(2000, 5, 17, 15, 4, 5, DateTimeKind.Utc);
 
         /// <summary>
         /// Fixture set up.
@@ -111,7 +111,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
 
             roleCache[new RoleKey(1, 101)] = new Role {Name = "Role_1", Date = StartDateTime};
             roleCache[new RoleKey(2, 102)] = new Role {Name = "Role_2", Date = StartDateTime.AddYears(1)};
-            roleCache[new RoleKey(3, 103)] = new Role {Name = null, Date = StartDateTime.AddYears(2)};
+            roleCache[new RoleKey(3, 103)] = new Role {Name = null, Date = StartDateTime.AddHours(5432)};
         }
 
         /// <summary>
@@ -123,7 +123,8 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             {
                 BinaryConfiguration = new BinaryConfiguration(typeof(Person),
                     typeof(Organization), typeof(Address), typeof(Role), typeof(RoleKey), typeof(Numerics)),
-                GridName = gridName
+                GridName = gridName,
+                JvmOptions = { "-Duser.timezone=UTC" }
             };
         }
 
@@ -735,7 +736,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
 
             // Test retrieval
             var dates = roles.OrderBy(x => x.Value.Date).Select(x => x.Value.Date);
-            var expDates = new[] {StartDateTime, StartDateTime.AddYears(1), StartDateTime.AddYears(2)};
+            var expDates = GetRoleCache().Select(x => x.Value.Date).OrderBy(x => x).ToArray();
             Assert.AreEqual(expDates, dates.ToArray());
 
             // Filtering
@@ -748,10 +749,21 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
                 join person in persons on role.Value.Date equals person.Value.Birthday
                 select person;
 
-            Assert.AreEqual(RoleCount, join.Count());
+            Assert.AreEqual(2, join.Count());
 
             // Functions
-            Assert.AreEqual("01 01 2000 00:00:00", dates.Select(x => x.ToString("DD MM YYYY HH:mm:ss")).First());
+            var strings = dates.Select(x => x.ToString("dd MM YYYY HH:mm:ss")).ToArray();
+            Assert.AreEqual(new[] {"17 05 2000 15:04:05", "29 12 2000 23:04:05", "17 05 2001 15:04:05"}, strings);
+
+            // Properties
+            Assert.AreEqual(new[] {2000, 2000, 2001}, dates.Select(x => x.Year).ToArray());
+            Assert.AreEqual(new[] {5, 12, 5}, dates.Select(x => x.Month).ToArray());
+            Assert.AreEqual(new[] {17, 29, 17}, dates.Select(x => x.Day).ToArray());
+            Assert.AreEqual(expDates.Select(x => x.DayOfYear).ToArray(), dates.Select(x => x.DayOfYear).ToArray());
+            Assert.AreEqual(expDates.Select(x => x.DayOfWeek).ToArray(), dates.Select(x => x.DayOfWeek).ToArray());
+            Assert.AreEqual(new[] {15, 23, 15}, dates.Select(x => x.Hour).ToArray());
+            Assert.AreEqual(new[] { 4, 4, 4 }, dates.Select(x => x.Minute).ToArray());
+            Assert.AreEqual(new[] { 5, 5, 5 }, dates.Select(x => x.Second).ToArray());
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bae8589/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
index 8231053..1f9da1c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/CacheQueryExpressionVisitor.cs
@@ -276,17 +276,9 @@ namespace Apache.Ignite.Linq.Impl
         {
             // Field hierarchy is flattened (Person.Address.Street is just Street), append as is, do not call Visit.
 
-            // Special case: string.Length
-            if (expression.Member == MethodVisitor.StringLength)
-            {
-                ResultBuilder.Append("length(");
-
-                VisitMember((MemberExpression) expression.Expression);
-
-                ResultBuilder.Append(")");
-
+            // Property call (string.Length, DateTime.Month, etc).
+            if (MethodVisitor.VisitPropertyCall(expression, this))
                 return expression;
-            }
 
             // Special case: grouping
             if (VisitGroupByMember(expression.Expression))

http://git-wip-us.apache.org/repos/asf/ignite/blob/3bae8589/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
index 3c8acc7..e83c448 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
@@ -29,8 +29,19 @@ namespace Apache.Ignite.Linq.Impl
     /// </summary>
     internal static class MethodVisitor
     {
-        /// <summary> The string length method. </summary>
-        public static readonly MemberInfo StringLength = typeof (string).GetProperty("Length");
+        /// <summary> Property visitors. </summary>
+        private static readonly Dictionary<MemberInfo, string> Properties = new Dictionary<MemberInfo, string>
+        {
+            {typeof(string).GetProperty("Length"), "length"},
+            {typeof(DateTime).GetProperty("Year"), "year"},
+            {typeof(DateTime).GetProperty("Month"), "month"},
+            {typeof(DateTime).GetProperty("Day"), "day_of_month"},
+            {typeof(DateTime).GetProperty("DayOfYear"), "day_of_year"},
+            {typeof(DateTime).GetProperty("DayOfWeek"), "-1 + day_of_week"},
+            {typeof(DateTime).GetProperty("Hour"), "hour"},
+            {typeof(DateTime).GetProperty("Minute"), "minute"},
+            {typeof(DateTime).GetProperty("Second"), "second"}
+        };
 
         /// <summary> Method visit delegate. </summary>
         private delegate void VisitMethodDelegate(MethodCallExpression expression, CacheQueryExpressionVisitor visitor);
@@ -103,6 +114,25 @@ namespace Apache.Ignite.Linq.Impl
         }.ToDictionary(x => x.Key, x => x.Value);
 
         /// <summary>
+        /// Visits the property call expression.
+        /// </summary>
+        public static bool VisitPropertyCall(MemberExpression expression, CacheQueryExpressionVisitor visitor)
+        {
+            string funcName;
+
+            if (!Properties.TryGetValue(expression.Member, out funcName))
+                return false;
+
+            visitor.ResultBuilder.Append(funcName).Append('(');
+
+            visitor.Visit(expression.Expression);
+
+            visitor.ResultBuilder.Append(')');
+
+            return true;
+        }
+
+        /// <summary>
         /// Visits the method call expression.
         /// </summary>
         public static void VisitMethodCall(MethodCallExpression expression, CacheQueryExpressionVisitor visitor)


[11/50] [abbrv] ignite git commit: IGNITE-1443: Implemented ContinuousQuery for C++

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/cache/cache.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/cache.h b/modules/platforms/cpp/core/include/ignite/cache/cache.h
index a975be3..54c0f96 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/cache.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/cache.h
@@ -37,6 +37,8 @@
 #include "ignite/cache/query/query_sql.h"
 #include "ignite/cache/query/query_text.h"
 #include "ignite/cache/query/query_sql_fields.h"
+#include "ignite/cache/query/continuous/continuous_query_handle.h"
+#include "ignite/cache/query/continuous/continuous_query.h"
 #include "ignite/impl/cache/cache_impl.h"
 #include "ignite/impl/operations.h"
 
@@ -1339,6 +1341,106 @@ namespace ignite
             }
 
             /**
+             * Start continuous query execution.
+             *
+             * @param qry Continuous query.
+             * @return Continuous query handle.
+             */
+            query::continuous::ContinuousQueryHandle<K, V> QueryContinuous(
+                const query::continuous::ContinuousQuery<K, V>& qry)
+            {
+                IgniteError err;
+
+                query::continuous::ContinuousQueryHandle<K, V> res = QueryContinuous(qry, err);
+
+                IgniteError::ThrowIfNeeded(err);
+
+                return res;
+            }
+
+            /**
+             * Start continuous query execution.
+             *
+             * @param qry Continuous query.
+             * @param err Error.
+             * @return Continuous query handle.
+             */
+            query::continuous::ContinuousQueryHandle<K, V> QueryContinuous(
+                const query::continuous::ContinuousQuery<K, V>& qry, IgniteError& err)
+            {
+                using namespace impl::cache::query::continuous;
+
+                if (!qry.impl.IsValid() || !qry.impl.Get()->HasListener())
+                {
+                    err = IgniteError(IgniteError::IGNITE_ERR_GENERIC,
+                        "Event listener is not set for ContinuousQuery instance");
+
+                    return query::continuous::ContinuousQueryHandle<K, V>();
+                }
+
+                ContinuousQueryHandleImpl* cqImpl;
+                cqImpl = impl.Get()->QueryContinuous(qry.impl, err);
+
+                if (cqImpl)
+                    cqImpl->SetQuery(qry.impl);
+
+                return query::continuous::ContinuousQueryHandle<K, V>(cqImpl);
+            }
+
+            /**
+             * Start continuous query execution with the initial query.
+             *
+             * @param qry Continuous query.
+             * @param initialQry Initial query to be executed.
+             * @return Continuous query handle.
+             */
+            template<typename Q>
+            query::continuous::ContinuousQueryHandle<K, V> QueryContinuous(
+                const query::continuous::ContinuousQuery<K, V>& qry,
+                const Q& initialQry)
+            {
+                IgniteError err;
+
+                query::continuous::ContinuousQueryHandle<K, V> res = QueryContinuous(qry, initialQry, err);
+
+                IgniteError::ThrowIfNeeded(err);
+
+                return res;
+            }
+
+            /**
+             * Start continuous query execution with the initial query.
+             *
+             * @param qry Continuous query.
+             * @param initialQry Initial query to be executed.
+             * @param err Error.
+             * @return Continuous query handle.
+             */
+            template<typename Q>
+            query::continuous::ContinuousQueryHandle<K, V> QueryContinuous(
+                const query::continuous::ContinuousQuery<K, V>& qry,
+                const Q& initialQry, IgniteError& err)
+            {
+                using namespace impl::cache::query::continuous;
+
+                if (!qry.impl.IsValid() || !qry.impl.Get()->HasListener())
+                {
+                    err = IgniteError(IgniteError::IGNITE_ERR_GENERIC,
+                        "Event listener is not set for ContinuousQuery instance");
+
+                    return query::continuous::ContinuousQueryHandle<K, V>();
+                }
+
+                ContinuousQueryHandleImpl* cqImpl;
+                cqImpl = impl.Get()->QueryContinuous(qry.impl, initialQry, err);
+
+                if (cqImpl)
+                    cqImpl->SetQuery(qry.impl);
+
+                return query::continuous::ContinuousQueryHandle<K, V>(cqImpl);
+            }
+
+            /**
              * Check if the instance is valid.
              *
              * Invalid instance can be returned if some of the previous
@@ -1356,7 +1458,7 @@ namespace ignite
 
         private:
             /** Implementation delegate. */
-            ignite::common::concurrent::SharedPointer<impl::cache::CacheImpl> impl;
+            common::concurrent::SharedPointer<impl::cache::CacheImpl> impl;
         };
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/cache/cache_entry.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/cache_entry.h b/modules/platforms/cpp/core/include/ignite/cache/cache_entry.h
index c737940..aea5182 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/cache_entry.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/cache_entry.h
@@ -45,7 +45,9 @@ namespace ignite
              * Creates instance with both key and value default-constructed.
              */
             CacheEntry() :
-                key(), val()
+                key(),
+                val(),
+                hasValue(false)
             {
                 // No-op.
             }
@@ -57,7 +59,9 @@ namespace ignite
              * @param val Value.
              */
             CacheEntry(const K& key, const V& val) :
-                key(key), val(val)
+                key(key),
+                val(val),
+                hasValue(true)
             {
                 // No-op.
             }
@@ -68,7 +72,17 @@ namespace ignite
              * @param other Other instance.
              */
             CacheEntry(const CacheEntry& other) :
-                key(other.key), val(other.val)
+                key(other.key),
+                val(other.val),
+                hasValue(other.hasValue)
+            {
+                // No-op.
+            }
+
+            /**
+             * Destructor.
+             */
+            virtual ~CacheEntry()
             {
                 // No-op.
             }
@@ -84,6 +98,7 @@ namespace ignite
                 {
                     key = other.key;
                     val = other.val;
+                    hasValue = other.hasValue;
                 }
 
                 return *this;
@@ -94,7 +109,7 @@ namespace ignite
              *
              * @return Key.
              */
-            K GetKey() const
+            const K& GetKey() const
             {
                 return key;
             }
@@ -104,17 +119,30 @@ namespace ignite
              *
              * @return Value.
              */
-            V GetValue() const
+            const V& GetValue() const
             {
                 return val;
             }
 
-        private:
+            /**
+             * Check if the value exists.
+             *
+             * @return True, if the value exists.
+             */
+            bool HasValue() const
+            {
+                return hasValue;
+            }
+
+        protected:
             /** Key. */
             K key;
 
             /** Value. */
             V val;
+
+            /** Indicates whether value exists */
+            bool hasValue;
         };
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/cache/event/cache_entry_event.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/event/cache_entry_event.h b/modules/platforms/cpp/core/include/ignite/cache/event/cache_entry_event.h
new file mode 100644
index 0000000..14fa185
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/cache/event/cache_entry_event.h
@@ -0,0 +1,139 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::cache::event::CacheEntryEvent class.
+ */
+
+#ifndef _IGNITE_CACHE_EVENT_CACHE_ENTRY_EVENT
+#define _IGNITE_CACHE_EVENT_CACHE_ENTRY_EVENT
+
+#include <ignite/binary/binary_raw_reader.h>
+#include <ignite/cache/cache_entry.h>
+
+namespace ignite
+{
+    namespace cache
+    {
+        /**
+         * Cache entry event class template.
+         *
+         * Both key and value types should be default-constructable,
+         * copy-constructable and assignable.
+         */
+        template<typename K, typename V>
+        class CacheEntryEvent : public CacheEntry<K, V>
+        {
+        public:
+            /**
+             * Default constructor.
+             *
+             * Creates instance with all fields default-constructed.
+             */
+            CacheEntryEvent() :
+                CacheEntry<K, V>(),
+                oldVal(),
+                hasOldValue(false)
+            {
+                // No-op.
+            }
+
+            /**
+             * Copy constructor.
+             *
+             * @param other Other instance.
+             */
+            CacheEntryEvent(const CacheEntryEvent<K, V>& other) :
+                CacheEntry<K, V>(other),
+                oldVal(other.oldVal),
+                hasOldValue(other.hasOldValue)
+            {
+                // No-op.
+            }
+
+            /**
+             * Destructor.
+             */
+            virtual ~CacheEntryEvent()
+            {
+                // No-op.
+            }
+
+            /**
+             * Assignment operator.
+             *
+             * @param other Other instance.
+             * @return *this.
+             */
+            CacheEntryEvent& operator=(const CacheEntryEvent<K, V>& other)
+            {
+                if (this != &other)
+                {
+                    CacheEntry<K, V>::operator=(other);
+
+                    oldVal = other.oldVal;
+                    hasOldValue = other.hasOldValue;
+                }
+
+                return *this;
+            }
+
+            /**
+             * Get old value.
+             *
+             * @return Old value.
+             */
+            const V& GetOldValue() const
+            {
+                return oldVal;
+            }
+
+            /**
+             * Check if the old value exists.
+             *
+             * @return True, if the old value exists.
+             */
+            bool HasOldValue() const
+            {
+                return hasOldValue;
+            }
+
+            /**
+             * Reads cache event using provided raw reader.
+             *
+             * @param reader Reader to use.
+             */
+            void Read(binary::BinaryRawReader& reader)
+            {
+                this->key = reader.ReadObject<K>();
+
+                this->hasOldValue = reader.TryReadObject(this->oldVal);
+                this->hasValue = reader.TryReadObject(this->val);
+            }
+
+        private:
+            /** Old value. */
+            V oldVal;
+
+            /** Indicates whether old value exists */
+            bool hasOldValue;
+        };
+    }
+}
+
+#endif //_IGNITE_CACHE_EVENT_CACHE_ENTRY_EVENT

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/cache/event/cache_entry_event_listener.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/event/cache_entry_event_listener.h b/modules/platforms/cpp/core/include/ignite/cache/event/cache_entry_event_listener.h
new file mode 100644
index 0000000..dd8f4a2
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/cache/event/cache_entry_event_listener.h
@@ -0,0 +1,71 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::cache::event::CacheEntryEventListener class.
+ */
+
+#ifndef _IGNITE_CACHE_EVENT_CACHE_ENTRY_EVENT_LISTENER
+#define _IGNITE_CACHE_EVENT_CACHE_ENTRY_EVENT_LISTENER
+
+#include <stdint.h>
+
+#include <ignite/cache/event/cache_entry_event.h>
+
+namespace ignite
+{
+    namespace cache
+    {
+        namespace event
+        {
+            /**
+             * Cache entry event listener.
+             */
+            template<typename K, typename V>
+            class CacheEntryEventListener
+            {
+            public:
+                /**
+                 * Default constructor.
+                 */
+                CacheEntryEventListener()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Destructor.
+                 */
+                virtual ~CacheEntryEventListener()
+                {
+                    // No-op.
+                }
+
+                /**
+                 * Event callback.
+                 *
+                 * @param evts Events.
+                 * @param num Events number.
+                 */
+                virtual void OnEvent(const CacheEntryEvent<K, V>* evts, uint32_t num) = 0;
+            };
+        }
+    }
+}
+
+#endif //_IGNITE_CACHE_EVENT_CACHE_ENTRY_EVENT_LISTENER
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h b/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h
new file mode 100644
index 0000000..563b11a
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h
@@ -0,0 +1,239 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::cache::query::continuous::ContinuousQuery class.
+ */
+
+#ifndef _IGNITE_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY
+#define _IGNITE_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY
+
+#include <ignite/impl/cache/query/continuous/continuous_query_impl.h>
+#include <ignite/cache/event/cache_entry_event_listener.h>
+
+namespace ignite
+{
+    namespace cache
+    {
+        // Forward-declaration.
+        template<typename K, typename V>
+        class IGNITE_IMPORT_EXPORT Cache;
+
+        namespace query
+        {
+            namespace continuous
+            {
+                /**
+                 * Continuous query.
+                 *
+                 * Continuous queries allow to register a remote and a listener
+                 * for cache update events. On any update to the related cache
+                 * an event is sent to the node that has executed the query and
+                 * listener is notified on that node.
+                 *
+                 * Continuous query can either be executed on the whole topology
+                 * or only on local node.
+                 *
+                 * To execute the query over the cache use method
+                 * ignite::cache::Cache::QueryContinuous().
+                 */
+                template<typename K, typename V>
+                class ContinuousQuery
+                {
+                    friend class Cache<K, V>;
+                public:
+
+                    /**
+                     * Default value for the buffer size.
+                     */
+                    enum { DEFAULT_BUFFER_SIZE = 1 };
+
+                    /**
+                     * Default value for the time interval.
+                     */
+                    enum { DEFAULT_TIME_INTERVAL = 0 };
+
+                    /**
+                     * Destructor.
+                     */
+                    ~ContinuousQuery()
+                    {
+                        // No-op.
+                    }
+
+                    /**
+                     * Constructor.
+                     *
+                     * @param lsnr Event listener. Invoked on the node where
+                     *     continuous query execution has been started.
+                     */
+                    ContinuousQuery(Reference<event::CacheEntryEventListener<K, V>> lsnr) :
+                        impl(new impl::cache::query::continuous::ContinuousQueryImpl<K, V>(lsnr))
+                    {
+                        // No-op.
+                    }
+
+                    /**
+                     * Constructor.
+                     *
+                     * @param lsnr Event listener Invoked on the node where
+                     *     continuous query execution has been started.
+                     * @param loc Whether query should be executed locally.
+                     */
+                    ContinuousQuery(Reference<event::CacheEntryEventListener<K, V>> lsnr, bool loc) :
+                        impl(new impl::cache::query::continuous::ContinuousQueryImpl<K, V>(lsnr, loc))
+                    {
+                        // No-op.
+                    }
+
+                    /**
+                     * Set local flag.
+                     *
+                     * @param val Value of the flag. If true, query will be
+                     *     executed only on local node, so only local entries
+                     *     will be returned as query result.
+                     */
+                    void SetLocal(bool val)
+                    {
+                        impl.Get()->SetLocal(val);
+                    }
+
+                    /**
+                     * Get local flag.
+                     *
+                     * @return Value of the flag. If true, query will be
+                     *     executed only on local node, so only local entries
+                     *     will be returned as query result.
+                     */
+                    bool GetLocal() const
+                    {
+                        return impl.Get()->GetLocal();
+                    }
+
+                    /**
+                     * Set buffer size.
+                     *
+                     * When a cache update happens, entry is first
+                     * put into a buffer. Entries from buffer will be sent to
+                     * the master node only if the buffer is full or time
+                     * provided via timeInterval is exceeded.
+                     *
+                     * @param val Buffer size.
+                     */
+                    void SetBufferSize(int32_t val)
+                    {
+                        impl.Get()->SetBufferSize(val);
+                    }
+
+                    /**
+                     * Get buffer size.
+                     *
+                     * When a cache update happens, entry is first
+                     * put into a buffer. Entries from buffer will be sent to
+                     * the master node only if the buffer is full or time
+                     * provided via timeInterval is exceeded.
+                     *
+                     * @return Buffer size.
+                     */
+                    int32_t GetBufferSize() const
+                    {
+                        return impl.Get()->GetBufferSize();
+                    }
+
+                    /**
+                     * Set time interval.
+                     *
+                     * When a cache update happens, entry is first put into
+                     * a buffer. Entries from buffer are sent to the master node
+                     * only if the buffer is full (its size can be changed via
+                     * SetBufferSize) or time provided via this method is
+                     * exceeded.
+                     *
+                     * Default value is DEFAULT_TIME_INTERVAL, i.e. 0, which
+                     * means that time check is disabled and entries will be
+                     * sent only when buffer is full.
+                     *
+                     * @param val Time interval in miliseconds.
+                     */
+                    void SetTimeInterval(int64_t val)
+                    {
+                        impl.Get()->SetTimeInterval(val);
+                    }
+
+                    /**
+                     * Get time interval.
+                     *
+                     * When a cache update happens, entry is first put into
+                     * a buffer. Entries from buffer are sent to the master node
+                     * only if the buffer is full (its size can be changed via
+                     * SetBufferSize) or time provided via SetTimeInterval
+                     * method is exceeded.
+                     *
+                     * Default value is DEFAULT_TIME_INTERVAL, i.e. 0, which
+                     * means that time check is disabled and entries will be
+                     * sent only when buffer is full.
+                     *
+                     * @return Time interval.
+                     */
+                    int64_t GetTimeInterval() const
+                    {
+                        return impl.Get()->GetTimeInterval();
+                    }
+
+                    /**
+                     * Set cache entry event listener.
+                     *
+                     * @param val Cache entry event listener. Invoked on the
+                     *     node where continuous query execution has been
+                     *     started.
+                     */
+                    void SetListener(Reference<event::CacheEntryEventListener<K, V>> lsnr)
+                    {
+                        impl.Get()->SetListener(val);
+                    }
+
+                    /**
+                     * Get cache entry event listener.
+                     *
+                     * @return Cache entry event listener.
+                     */
+                    const event::CacheEntryEventListener<K, V>& GetListener() const
+                    {
+                        return impl.Get()->GetListener();
+                    }
+
+                    /**
+                     * Get cache entry event listener.
+                     *
+                     * @return Cache entry event listener.
+                     */
+                    event::CacheEntryEventListener<K, V>& GetListener()
+                    {
+                        return impl.Get()->GetListener();
+                    }
+
+                private:
+                    /** Implementation. */
+                    common::concurrent::SharedPointer<impl::cache::query::continuous::ContinuousQueryImpl<K, V>> impl;
+                };
+            }
+        }
+    }
+}
+
+#endif //_IGNITE_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query_handle.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query_handle.h b/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query_handle.h
new file mode 100644
index 0000000..bbefbcc
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query_handle.h
@@ -0,0 +1,133 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::cache::query::continuous::ContinuousQueryHandle class.
+ */
+
+#ifndef _IGNITE_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY_HANDLE
+#define _IGNITE_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY_HANDLE
+
+#include <ignite/impl/cache/query/continuous/continuous_query_handle_impl.h>
+
+namespace ignite
+{
+    namespace cache
+    {
+        namespace query
+        {
+            namespace continuous
+            {
+                /**
+                 * Continuous query handle.
+                 */
+                template<typename K, typename V>
+                class ContinuousQueryHandle
+                {
+                public:
+                    /**
+                     * Default constructor.
+                     */
+                    ContinuousQueryHandle() :
+                        impl()
+                    {
+                        // No-op.
+                    }
+
+                    /**
+                     * Constructor.
+                     *
+                     * Internal method. Should not be used by user.
+                     *
+                     * @param impl Implementation.
+                     */
+                    ContinuousQueryHandle(impl::cache::query::continuous::ContinuousQueryHandleImpl* impl) :
+                        impl(impl)
+                    {
+                        // No-op.
+                    }
+
+                    /**
+                     * Gets the cursor for initial query.
+                     * Can be called only once, throws IgniteError on consequent
+                     * calls.
+                     *
+                     * @return Initial query cursor.
+                     */
+                    QueryCursor<K, V> GetInitialQueryCursor()
+                    {
+                        IgniteError err;
+
+                        QueryCursor<K, V> res = GetInitialQueryCursor(err);
+
+                        IgniteError::ThrowIfNeeded(err);
+
+                        return res;
+                    }
+
+                    /**
+                     * Gets the cursor for initial query.
+                     * Can be called only once, results in error on consequent
+                     * calls.
+                     *
+                     * @param err Error.
+                     * @return Initial query cursor.
+                     */
+                    QueryCursor<K, V> GetInitialQueryCursor(IgniteError& err)
+                    {
+                        impl::cache::query::continuous::ContinuousQueryHandleImpl* impl0 = impl.Get();
+
+                        if (impl0)
+                            return QueryCursor<K, V>(impl0->GetInitialQueryCursor(err));
+                        else
+                        {
+                            err = IgniteError(IgniteError::IGNITE_ERR_GENERIC,
+                                "Instance is not usable (did you check for error?).");
+
+                            return QueryCursor<K, V>();
+                        }
+                    }
+
+                    /**
+                     * Check if the instance is valid.
+                     *
+                     * Invalid instance can be returned if some of the previous
+                     * operations have resulted in a failure. For example invalid
+                     * instance can be returned by not-throwing version of method
+                     * in case of error. Invalid instances also often can be
+                     * created using default constructor.
+                     *
+                     * @return True if the instance is valid and can be used.
+                     */
+                    bool IsValid() const
+                    {
+                        return impl.IsValid();
+                    }
+
+                private:
+                    typedef impl::cache::query::continuous::ContinuousQueryHandleImpl ContinuousQueryHandleImpl;
+
+                    /** Implementation delegate. */
+                    common::concurrent::SharedPointer<ContinuousQueryHandleImpl> impl;
+                };
+            }
+        }
+    }
+}
+
+#endif //_IGNITE_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY_HANDLE
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/impl/cache/cache_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/cache/cache_impl.h b/modules/platforms/cpp/core/include/ignite/impl/cache/cache_impl.h
index 3e0f177..535e3ec 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/cache/cache_impl.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/cache/cache_impl.h
@@ -22,7 +22,10 @@
 #include <ignite/cache/query/query_sql.h>
 #include <ignite/cache/query/query_text.h>
 #include <ignite/cache/query/query_sql_fields.h>
+#include <ignite/cache/query/continuous/continuous_query_handle.h>
 #include <ignite/impl/cache/query/query_impl.h>
+#include <ignite/impl/cache/query/continuous/continuous_query_handle_impl.h>
+#include <ignite/impl/cache/query/continuous/continuous_query_impl.h>
 
 #include <ignite/impl/interop/interop_target.h>
 
@@ -309,12 +312,59 @@ namespace ignite
                  * @return Query cursor.
                  */
                 query::QueryCursorImpl* QuerySqlFields(const ignite::cache::query::SqlFieldsQuery& qry, IgniteError* err);
-                
+
+                /**
+                 * Start continuous query execution.
+                 *
+                 * @param qry Continuous query.
+                 * @param err Error.
+                 * @return Continuous query handle.
+                 */
+                query::continuous::ContinuousQueryHandleImpl* QueryContinuous(
+                    const common::concurrent::SharedPointer<query::continuous::ContinuousQueryImplBase> qry,
+                    IgniteError& err);
+
+                /**
+                 * Start continuous query execution with initial query.
+                 *
+                 * @param qry Continuous query.
+                 * @param initialQry Initial query.
+                 * @param err Error.
+                 * @return Continuous query handle.
+                 */
+                query::continuous::ContinuousQueryHandleImpl* QueryContinuous(
+                    const common::concurrent::SharedPointer<query::continuous::ContinuousQueryImplBase> qry,
+                    const ignite::cache::query::SqlQuery& initialQry, IgniteError& err);
+
+                /**
+                 * Start continuous query execution with initial query.
+                 *
+                 * @param qry Continuous query.
+                 * @param initialQry Initial query.
+                 * @param err Error.
+                 * @return Continuous query handle.
+                 */
+                query::continuous::ContinuousQueryHandleImpl* QueryContinuous(
+                    const common::concurrent::SharedPointer<query::continuous::ContinuousQueryImplBase> qry,
+                    const ignite::cache::query::TextQuery& initialQry, IgniteError& err);
+
+                /**
+                 * Start continuous query execution with initial query.
+                 *
+                 * @param qry Continuous query.
+                 * @param initialQry Initial query.
+                 * @param err Error.
+                 * @return Continuous query handle.
+                 */
+                query::continuous::ContinuousQueryHandleImpl* QueryContinuous(
+                    const common::concurrent::SharedPointer<query::continuous::ContinuousQueryImplBase> qry,
+                    const ignite::cache::query::ScanQuery& initialQry, IgniteError& err);
+
             private:
+                IGNITE_NO_COPY_ASSIGNMENT(CacheImpl)
+
                 /** Name. */
                 char* name; 
-                
-                IGNITE_NO_COPY_ASSIGNMENT(CacheImpl)
 
                 /**
                  * Internal query execution routine.
@@ -346,11 +396,67 @@ namespace ignite
                     if (jniErr.code == ignite::java::IGNITE_JNI_ERR_SUCCESS)
                         return new query::QueryCursorImpl(GetEnvironmentPointer(), qryJavaRef);
                     else
-                        return NULL;
+                        return 0;
+                }
+
+                /**
+                 * Start continuous query execution with the initial query.
+                 *
+                 * @param qry Continuous query.
+                 * @param initialQry Initial query to be executed.
+                 * @param err Error.
+                 * @return Continuous query handle.
+                 */
+                template<typename T>
+                query::continuous::ContinuousQueryHandleImpl* QueryContinuous(
+                    const common::concurrent::SharedPointer<query::continuous::ContinuousQueryImplBase> qry,
+                    const T& initialQry, int32_t typ, int32_t cmd, IgniteError& err)
+                {
+                    jni::java::JniErrorInfo jniErr;
+
+                    common::concurrent::SharedPointer<interop::InteropMemory> mem = GetEnvironment().AllocateMemory();
+                    interop::InteropMemory* mem0 = mem.Get();
+                    interop::InteropOutputStream out(mem0);
+                    binary::BinaryWriterImpl writer(&out, GetEnvironment().GetTypeManager());
+                    ignite::binary::BinaryRawWriter rawWriter(&writer);
+
+                    const query::continuous::ContinuousQueryImplBase& qry0 = *qry.Get();
+
+                    int64_t handle = GetEnvironment().GetHandleRegistry().Allocate(qry);
+
+                    rawWriter.WriteInt64(handle);
+                    rawWriter.WriteBool(qry0.GetLocal());
+
+                    // Filters are not supported for now.
+                    rawWriter.WriteBool(false);
+                    rawWriter.WriteNull();
+
+                    rawWriter.WriteInt32(qry0.GetBufferSize());
+                    rawWriter.WriteInt64(qry0.GetTimeInterval());
+
+                    // Autounsubscribe is a filter feature.
+                    rawWriter.WriteBool(false);
+
+                    // Writing initial query. When there is not initial query writing -1.
+                    rawWriter.WriteInt32(typ);
+                    if (typ != -1)
+                        initialQry.Write(rawWriter);
+
+                    out.Synchronize();
+
+                    jobject qryJavaRef = GetEnvironment().Context()->CacheOutOpContinuousQuery(GetTarget(),
+                        cmd, mem.Get()->PointerLong(), &jniErr);
+
+                    IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err);
+
+                    if (jniErr.code == java::IGNITE_JNI_ERR_SUCCESS)
+                        return new query::continuous::ContinuousQueryHandleImpl(GetEnvironmentPointer(), handle, qryJavaRef);
+
+                    return 0;
                 }
             };
         }
     }    
 }
 
-#endif
\ No newline at end of file
+#endif

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_handle_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_handle_impl.h b/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_handle_impl.h
new file mode 100644
index 0000000..75504b1
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_handle_impl.h
@@ -0,0 +1,101 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::impl::cache::query::continuous::ContinuousQueryHandleImpl class.
+ */
+
+#ifndef _IGNITE_IMPL_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY_HANDLE_IMPL
+#define _IGNITE_IMPL_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY_HANDLE_IMPL
+
+#include "ignite/cache/query/query_cursor.h"
+#include "ignite/impl/cache/query/continuous/continuous_query_impl.h"
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace cache
+        {
+            namespace query
+            {
+                namespace continuous
+                {
+                    /**
+                     * Continuous query handle implementation.
+                     */
+                    class IGNITE_IMPORT_EXPORT ContinuousQueryHandleImpl
+                    {
+                        typedef common::concurrent::SharedPointer<IgniteEnvironment> SP_IgniteEnvironment;
+                        typedef common::concurrent::SharedPointer<ContinuousQueryImplBase> SP_ContinuousQueryImplBase;
+                    public:
+                        /**
+                         * Default constructor.
+                         * 
+                         * @param env Environment.
+                         * @param javaRef Java reference.
+                         */
+                        ContinuousQueryHandleImpl(SP_IgniteEnvironment env, int64_t handle, jobject javaRef);
+
+                        /**
+                         * Destructor.
+                         */
+                        ~ContinuousQueryHandleImpl();
+
+                        /**
+                         * Gets the cursor for initial query.
+                         * Can be called only once, throws exception on consequent calls.
+                         *
+                         * @param err Error.
+                         * @return Initial query cursor.
+                         */
+                        QueryCursorImpl* GetInitialQueryCursor(IgniteError& err);
+
+                        /**
+                         * Set query to keep pointer to.
+                         *
+                         * @param query Query.
+                         */
+                        void SetQuery(SP_ContinuousQueryImplBase query);
+
+                    private:
+                        /** Environment. */
+                        SP_IgniteEnvironment env;
+
+                        /** Local handle for handle registry. */
+                        int64_t handle;
+
+                        /** Handle to Java object. */
+                        jobject javaRef;
+
+                        /** Shared pointer to query. Kept for query to live long enough. */
+                        SP_ContinuousQueryImplBase qry;
+
+                        /** Mutex. */
+                        common::concurrent::CriticalSection mutex;
+
+                        /** Cursor extracted. */
+                        bool extracted;
+                    };
+                }
+            }
+        }
+    }
+}
+
+#endif //_IGNITE_IMPL_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY_HANDLE_IMPL
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_impl.h b/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_impl.h
new file mode 100644
index 0000000..50ced12
--- /dev/null
+++ b/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_impl.h
@@ -0,0 +1,351 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::impl::cache::query::continuous::ContinuousQueryImpl class.
+ */
+
+#ifndef _IGNITE_IMPL_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY_IMPL
+#define _IGNITE_IMPL_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY_IMPL
+
+#include <stdint.h>
+
+#include <ignite/reference.h>
+
+#include <ignite/cache/event/cache_entry_event_listener.h>
+#include <ignite/binary/binary_raw_reader.h>
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace cache
+        {
+            namespace query
+            {
+                namespace continuous
+                {
+                    /**
+                     * Continuous query base implementation class.
+                     *
+                     * Continuous queries allow to register a remote and a listener
+                     * for cache update events. On any update to the related cache
+                     * an event is sent to the node that has executed the query and
+                     * listener is notified on that node.
+                     *
+                     * Continuous query can either be executed on the whole topology
+                     * or only on local node.
+                     *
+                     * To execute the query over the cache use method
+                     * ignite::cache::Cache::QueryContinuous().
+                     */
+                    class ContinuousQueryImplBase
+                    {
+                    public:
+                        /**
+                         * Destructor.
+                         */
+                        virtual ~ContinuousQueryImplBase()
+                        {
+                            // No-op.
+                        }
+
+                        /**
+                         * Default value for the buffer size.
+                         */
+                        enum { DEFAULT_BUFFER_SIZE = 1 };
+
+                        /**
+                         * Default value for the time interval.
+                         */
+                        enum { DEFAULT_TIME_INTERVAL = 0 };
+
+                        /**
+                         * Constructor.
+                         *
+                         * @param loc Whether query should be executed locally.
+                         */
+                        explicit ContinuousQueryImplBase(bool loc) :
+                            local(loc),
+                            bufferSize(DEFAULT_BUFFER_SIZE),
+                            timeInterval(DEFAULT_TIME_INTERVAL)
+                        {
+                            // No-op.
+                        }
+
+                        /**
+                         * Set local flag.
+                         *
+                         * @param val Value of the flag. If true, query will be
+                         *     executed only on local node, so only local entries
+                         *     will be returned as query result.
+                         */
+                        void SetLocal(bool val)
+                        {
+                            local = val;
+                        }
+
+                        /**
+                         * Get local flag.
+                         *
+                         * @return Value of the flag. If true, query will be
+                         *     executed only on local node, so only local entries
+                         *     will be returned as query result.
+                         */
+                        bool GetLocal() const
+                        {
+                            return local;
+                        }
+
+                        /**
+                         * Set buffer size.
+                         *
+                         * When a cache update happens, entry is first
+                         * put into a buffer. Entries from buffer will be sent to
+                         * the master node only if the buffer is full or time
+                         * provided via timeInterval is exceeded.
+                         *
+                         * @param val Buffer size.
+                         */
+                        void SetBufferSize(int32_t val)
+                        {
+                            bufferSize = val;
+                        }
+
+                        /**
+                         * Get buffer size.
+                         *
+                         * When a cache update happens, entry is first
+                         * put into a buffer. Entries from buffer will be sent to
+                         * the master node only if the buffer is full or time
+                         * provided via timeInterval is exceeded.
+                         *
+                         * @return Buffer size.
+                         */
+                        int32_t GetBufferSize() const
+                        {
+                            return bufferSize;
+                        }
+
+                        /**
+                         * Set time interval.
+                         *
+                         * When a cache update happens, entry is first put into
+                         * a buffer. Entries from buffer are sent to the master node
+                         * only if the buffer is full (its size can be changed via
+                         * SetBufferSize) or time provided via this method is
+                         * exceeded.
+                         *
+                         * Default value is DEFAULT_TIME_INTERVAL, i.e. 0, which
+                         * means that time check is disabled and entries will be
+                         * sent only when buffer is full.
+                         *
+                         * @param val Time interval in miliseconds.
+                         */
+                        void SetTimeInterval(int64_t val)
+                        {
+                            timeInterval = val;
+                        }
+
+                        /**
+                         * Get time interval.
+                         *
+                         * When a cache update happens, entry is first put into
+                         * a buffer. Entries from buffer are sent to the master node
+                         * only if the buffer is full (its size can be changed via
+                         * SetBufferSize) or time provided via SetTimeInterval
+                         * method is exceeded.
+                         *
+                         * Default value is DEFAULT_TIME_INTERVAL, i.e. 0, which
+                         * means that time check is disabled and entries will be
+                         * sent only when buffer is full.
+                         *
+                         * @return Time interval.
+                         */
+                        int64_t GetTimeInterval() const
+                        {
+                            return timeInterval;
+                        }
+
+                        /**
+                         * Callback that reads and processes cache events.
+                         *
+                         * @param reader Reader to use.
+                         */
+                        virtual void ReadAndProcessEvents(ignite::binary::BinaryRawReader& reader) = 0;
+
+                    private:
+                        /**
+                         * Local flag. When set query will be executed only on local
+                         * node, so only local entries will be returned as query
+                         * result.
+                         *
+                         * Default value is false.
+                         */
+                        bool local;
+
+                        /**
+                         * Buffer size. When a cache update happens, entry is first
+                         * put into a buffer. Entries from buffer will be sent to
+                         * the master node only if the buffer is full or time
+                         * provided via timeInterval is exceeded.
+                         *
+                         * Default value is DEFAULT_BUFFER_SIZE.
+                         */
+                        int32_t bufferSize;
+
+                        /**
+                         * Time interval in miliseconds. When a cache update
+                         * happens, entry is first put into a buffer. Entries from
+                         * buffer will be sent to the master node only if the buffer
+                         * is full (its size can be changed via SetBufferSize) or
+                         * time provided via SetTimeInterval method is exceeded.
+                         *
+                         * Default value is DEFAULT_TIME_INTERVAL, i.e. 0, which
+                         * means that time check is disabled and entries will be
+                         * sent only when buffer is full.
+                         */
+                        int64_t timeInterval;
+                    };
+
+                    /**
+                     * Continuous query implementation.
+                     *
+                     * Continuous queries allow to register a remote and a listener
+                     * for cache update events. On any update to the related cache
+                     * an event is sent to the node that has executed the query and
+                     * listener is notified on that node.
+                     *
+                     * Continuous query can either be executed on the whole topology
+                     * or only on local node.
+                     *
+                     * To execute the query over the cache use method
+                     * ignite::cache::Cache::QueryContinuous().
+                     */
+                    template<typename K, typename V>
+                    class ContinuousQueryImpl : public ContinuousQueryImplBase
+                    {
+                    public:
+                        /**
+                         * Destructor.
+                         */
+                        virtual ~ContinuousQueryImpl()
+                        {
+                            // No-op.
+                        }
+
+                        /**
+                         * Constructor.
+                         *
+                         * @param lsnr Event listener. Invoked on the node where
+                         *     continuous query execution has been started.
+                         */
+                        ContinuousQueryImpl(Reference<ignite::cache::event::CacheEntryEventListener<K, V>>& lsnr) :
+                            ContinuousQueryImplBase(false),
+                            lsnr(lsnr)
+                        {
+                            // No-op.
+                        }
+
+                        /**
+                         * Constructor.
+                         *
+                         * @param lsnr Event listener Invoked on the node where
+                         *     continuous query execution has been started.
+                         * @param loc Whether query should be executed locally.
+                         */
+                        ContinuousQueryImpl(Reference<ignite::cache::event::CacheEntryEventListener<K, V>>& lsnr, bool loc) :
+                            ContinuousQueryImplBase(loc),
+                            lsnr(lsnr)
+                        {
+                            // No-op.
+                        }
+
+                        /**
+                         * Set cache entry event listener.
+                         *
+                         * @param val Cache entry event listener. Invoked on the
+                         *     node where continuous query execution has been
+                         *     started.
+                         */
+                        void SetListener(Reference<ignite::cache::event::CacheEntryEventListener<K, V>>& val)
+                        {
+                            lsnr = val;
+                        }
+
+                        /**
+                         * Check if the query has listener.
+                         *
+                         * @return True if the query has listener.
+                         */
+                        bool HasListener() const
+                        {
+                            return !lsnr.IsNull();
+                        }
+
+                        /**
+                         * Get cache entry event listener.
+                         *
+                         * @return Cache entry event listener.
+                         */
+                        const ignite::cache::event::CacheEntryEventListener<K, V>& GetListener() const
+                        {
+                            return lsnr.Get();
+                        }
+
+                        /**
+                         * Get cache entry event listener.
+                         *
+                         * @return Cache entry event listener.
+                         */
+                        ignite::cache::event::CacheEntryEventListener<K, V>& GetListener()
+                        {
+                            return lsnr.Get();
+                        }
+
+                        /**
+                         * Callback that reads and processes cache events.
+                         *
+                         * @param reader Reader to use.
+                         */
+                        virtual void ReadAndProcessEvents(ignite::binary::BinaryRawReader& reader)
+                        {
+                            // Number of events.
+                            int32_t cnt = reader.ReadInt32();
+
+                            // Storing events here.
+                            std::vector< ignite::cache::CacheEntryEvent<K, V> > events;
+                            events.resize(cnt);
+
+                            for (int32_t i = 0; i < cnt; ++i)
+                                events[i].Read(reader);
+
+                            lsnr.Get().OnEvent(events.data(), cnt);
+                        }
+
+                    private:
+                        /** Cache entry event listener. */
+                        Reference<ignite::cache::event::CacheEntryEventListener<K, V>> lsnr;
+                    };
+                }
+            }
+        }
+    }
+}
+
+#endif //_IGNITE_IMPL_CACHE_QUERY_CONTINUOUS_CONTINUOUS_QUERY_IMPL
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/include/ignite/impl/handle_registry.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/handle_registry.h b/modules/platforms/cpp/core/include/ignite/impl/handle_registry.h
index 107042a..3c4d123 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/handle_registry.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/handle_registry.h
@@ -28,21 +28,9 @@ namespace ignite
     namespace impl
     {
         /**
-         * Something what can be registered inside handle registry.
-         */
-        class IGNITE_IMPORT_EXPORT HandleRegistryEntry
-        {
-        public:
-            /**
-             * Destructor.
-             */
-            virtual ~HandleRegistryEntry();
-        };
-
-        /**
          * Handle registry segment containing thread-specific data for slow-path access.
          */
-        class IGNITE_IMPORT_EXPORT HandleRegistrySegment
+        class HandleRegistrySegment
         {
         public:
             /**
@@ -61,7 +49,7 @@ namespace ignite
              * @param hnd Handle.
              * @return Associated entry or NULL if it doesn't exists.
              */
-            ignite::common::concurrent::SharedPointer<HandleRegistryEntry> Get(int64_t hnd);
+            common::concurrent::SharedPointer<void> Get(int64_t hnd);
 
             /**
              * Put entry into segment.
@@ -69,14 +57,14 @@ namespace ignite
              * @param hnd Handle.
              * @param entry Associated entry (cannot be NULL).
              */
-            void Put(int64_t hnd, const ignite::common::concurrent::SharedPointer<HandleRegistryEntry>& entry);
+            void Put(int64_t hnd, const common::concurrent::SharedPointer<void>& entry);
 
             /**
              * Remove entry from the segment.
              *
              * @param hnd Handle.
              */
-            void Remove(int64_t hnd);            
+            void Remove(int64_t hnd);
 
             /**
              * Clear all entries from the segment.
@@ -84,10 +72,10 @@ namespace ignite
             void Clear();
         private:
             /** Map with data. */
-            std::map<int64_t, ignite::common::concurrent::SharedPointer<HandleRegistryEntry>>* map;
+            std::map<int64_t, common::concurrent::SharedPointer<void>> map;
 
             /** Mutex. */
-            ignite::common::concurrent::CriticalSection* mux;
+            common::concurrent::CriticalSection mux;
 
             IGNITE_NO_COPY_ASSIGNMENT(HandleRegistrySegment);
         };
@@ -102,7 +90,7 @@ namespace ignite
              * Constructor.
              *
              * @param fastCap Fast-path capacity.
-             * @param segmentCnt Slow-path segments count.
+             * @param slowSegmentCnt Slow-path segments count.
              */
             HandleRegistry(int32_t fastCap, int32_t slowSegmentCnt);
 
@@ -117,7 +105,7 @@ namespace ignite
              * @param target Target.
              * @return Handle.
              */
-            int64_t Allocate(const ignite::common::concurrent::SharedPointer<HandleRegistryEntry>& target);
+            int64_t Allocate(const common::concurrent::SharedPointer<void>& target);
 
             /**
              * Allocate handle in critical mode.
@@ -125,7 +113,7 @@ namespace ignite
              * @param target Target.
              * @return Handle.
              */
-            int64_t AllocateCritical(const ignite::common::concurrent::SharedPointer<HandleRegistryEntry>& target);
+            int64_t AllocateCritical(const common::concurrent::SharedPointer<void>& target);
 
             /**
              * Allocate handle in safe mode.
@@ -133,7 +121,7 @@ namespace ignite
              * @param target Target.
              * @return Handle.
              */
-            int64_t AllocateSafe(const ignite::common::concurrent::SharedPointer<HandleRegistryEntry>& target);
+            int64_t AllocateSafe(const common::concurrent::SharedPointer<void>& target);
 
             /**
              * Allocate handle in critical and safe modes.
@@ -141,7 +129,7 @@ namespace ignite
              * @param target Target.
              * @return Handle.
              */
-            int64_t AllocateCriticalSafe(const ignite::common::concurrent::SharedPointer<HandleRegistryEntry>& target);
+            int64_t AllocateCriticalSafe(const common::concurrent::SharedPointer<void>& target);
 
             /**
              * Release handle.
@@ -154,35 +142,36 @@ namespace ignite
              * Get target.
              *
              * @param hnd Handle.
-             * @param Target.
+             * @return Target.
              */
-            ignite::common::concurrent::SharedPointer<HandleRegistryEntry> Get(int64_t hnd);
+            common::concurrent::SharedPointer<void> Get(int64_t hnd);
 
             /**
              * Close the registry.
              */
             void Close();
+
         private:
             /** Fast-path container capacity. */
-            int32_t fastCap;                     
+            int32_t fastCap;
 
             /** Fast-path counter. */
-            int32_t fastCtr;               
+            int32_t fastCtr;
 
             /** Fast-path container. */
-            ignite::common::concurrent::SharedPointer<HandleRegistryEntry>* fast;
+            common::concurrent::SharedPointer<void>* fast;
 
             /** Amount of slow-path segments. */
-            int32_t slowSegmentCnt;            
+            int32_t slowSegmentCnt;
 
             /** Slow-path counter. */
-            int64_t slowCtr;                                                         
-            
+            int64_t slowCtr;
+
             /** Slow-path segments. */
-            HandleRegistrySegment** slow;                                            
+            HandleRegistrySegment** slow;
 
             /** Close flag. */
-            int32_t closed;                                                           
+            int32_t closed;
 
             IGNITE_NO_COPY_ASSIGNMENT(HandleRegistry);
 
@@ -190,11 +179,10 @@ namespace ignite
              * Internal allocation routine.
              *
              * @param target Target.
-             * @param Critical mode flag.
-             * @param Safe mode flag.
+             * @param critical mode flag.
+             * @param safe mode flag.
              */
-            int64_t Allocate0(const ignite::common::concurrent::SharedPointer<HandleRegistryEntry>& target,
-                bool critical, bool safe);
+            int64_t Allocate0(const common::concurrent::SharedPointer<void>& target, bool critical, bool safe);
         };
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/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 fb6f657..2b2a117 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/ignite_environment.h
@@ -20,14 +20,15 @@
 
 #include <ignite/common/concurrent.h>
 #include <ignite/jni/java.h>
+#include <ignite/jni/utils.h>
 
 #include "ignite/impl/interop/interop_memory.h"
 #include "ignite/impl/binary/binary_type_manager.h"
-#include "ignite/jni/utils.h"
+#include "ignite/impl/handle_registry.h"
 
-namespace ignite 
+namespace ignite
 {
-    namespace impl 
+    namespace impl
     {
         /**
          * Defines environment in which Ignite operates.
@@ -41,6 +42,16 @@ namespace ignite
             enum { DEFAULT_ALLOCATION_SIZE = 1024 };
 
             /**
+             * Default fast path handle registry containers capasity.
+             */
+            enum { DEFAULT_FAST_PATH_CONTAINERS_CAP = 1024 };
+
+            /**
+            * Default slow path handle registry containers capasity.
+            */
+            enum { DEFAULT_SLOW_PATH_CONTAINERS_CAP = 16 };
+
+            /**
              * Default constructor.
              */
             IgniteEnvironment();
@@ -78,6 +89,13 @@ namespace ignite
             void OnStartCallback(long long memPtr, jobject proc);
 
             /**
+             * Continuous query listener apply callback.
+             *
+             * @param mem Memory with data.
+             */
+            void OnContinuousQueryListenerApply(common::concurrent::SharedPointer<interop::InteropMemory>& mem);
+
+            /**
              * Get name of Ignite instance.
              *
              * @return Name.
@@ -133,6 +151,13 @@ namespace ignite
              */
             void ProcessorReleaseStart();
 
+            /**
+             * Get handle registry.
+             *
+             * @return Handle registry.
+             */
+            HandleRegistry& GetHandleRegistry();
+
         private:
             /** Context to access Java. */
             common::concurrent::SharedPointer<jni::java::JniContext> ctx;
@@ -152,6 +177,9 @@ namespace ignite
             /** Type updater. */
             binary::BinaryTypeUpdater* metaUpdater;
 
+            /** Handle registry. */
+            HandleRegistry registry;
+
             IGNITE_NO_COPY_ASSIGNMENT(IgniteEnvironment);
         };
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/namespaces.dox
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/namespaces.dox b/modules/platforms/cpp/core/namespaces.dox
index 0f5f11f..49379e6 100644
--- a/modules/platforms/cpp/core/namespaces.dox
+++ b/modules/platforms/cpp/core/namespaces.dox
@@ -22,40 +22,54 @@
  * computing and transacting on large-scale data sets in real-time, orders of magnitude faster than possible with
  * traditional disk-based or flash-based technologies.
  */
- 
+
  /**
   * Apache %Ignite API.
   */
  namespace ignite
  {
-	 /**
-	  * %Ignite Binary Objects API.
-	  */
-	 namespace binary
-	 {
-		 // Empty.
-	 }
+     /**
+      * %Ignite Binary Objects API.
+      */
+     namespace binary
+     {
+         // Empty.
+     }
+
+     /**
+      * %Ignite %Transaction API.
+      */
+     namespace transactions
+     {
+         // Empty.
+     }
 
-	 /**
-	  * %Ignite %Transaction API.
-	  */
-	 namespace transactions
-	 {
-		 // Empty.
-	 }
-	 
-	 /**
-	  * %Ignite %Cache API.
-	  */
-	 namespace cache
-	 {
-		 /**
-		  * Contains APIs for creating and executing cache queries.
-		  */
-		 namespace query
-		 {
-			 // Empty.
-		 }
-	 }
+     /**
+      * %Ignite %Cache API.
+      */
+     namespace cache
+     {
+        /**
+         * Contains APIs for cache events.
+         */
+        namespace event
+        {
+            // Empty.
+        }
+
+         /**
+          * Contains APIs for creating and executing cache queries.
+          */
+         namespace query
+         {
+            /**
+             * Contains APIs for continuous queries.
+             */
+            namespace continuous
+            {
+                // Empty.
+            }
+         }
+     }
  }
- 
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/project/vs/core.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/project/vs/core.vcxproj b/modules/platforms/cpp/core/project/vs/core.vcxproj
index 6320323..89a2dff 100644
--- a/modules/platforms/cpp/core/project/vs/core.vcxproj
+++ b/modules/platforms/cpp/core/project/vs/core.vcxproj
@@ -193,6 +193,10 @@
     <ClInclude Include="..\..\include\ignite\cache\cache.h" />
     <ClInclude Include="..\..\include\ignite\cache\cache_entry.h" />
     <ClInclude Include="..\..\include\ignite\cache\cache_peek_mode.h" />
+    <ClInclude Include="..\..\include\ignite\cache\event\cache_entry_event.h" />
+    <ClInclude Include="..\..\include\ignite\cache\event\cache_entry_event_listener.h" />
+    <ClInclude Include="..\..\include\ignite\cache\query\continuous\continuous_query.h" />
+    <ClInclude Include="..\..\include\ignite\cache\query\continuous\continuous_query_handle.h" />
     <ClInclude Include="..\..\include\ignite\cache\query\query.h" />
     <ClInclude Include="..\..\include\ignite\cache\query\query_argument.h" />
     <ClInclude Include="..\..\include\ignite\cache\query\query_cursor.h" />
@@ -208,6 +212,8 @@
     <ClInclude Include="..\..\include\ignite\impl\binary\binary_type_updater_impl.h" />
     <ClInclude Include="..\..\include\ignite\impl\cache\cache_impl.h" />
     <ClInclude Include="..\..\include\ignite\impl\cache\query\query_batch.h" />
+    <ClInclude Include="..\..\include\ignite\impl\cache\query\continuous\continuous_query_handle_impl.h" />
+    <ClInclude Include="..\..\include\ignite\impl\cache\query\continuous\continuous_query_impl.h" />
     <ClInclude Include="..\..\include\ignite\impl\cache\query\query_fields_row_impl.h" />
     <ClInclude Include="..\..\include\ignite\impl\cache\query\query_impl.h" />
     <ClInclude Include="..\..\include\ignite\impl\ignite_environment.h" />
@@ -229,6 +235,7 @@
     <ClCompile Include="..\..\src\impl\binary\binary_type_updater_impl.cpp" />
     <ClCompile Include="..\..\src\impl\cache\cache_impl.cpp" />
     <ClCompile Include="..\..\src\impl\cache\query\query_batch.cpp" />
+    <ClCompile Include="..\..\src\impl\cache\query\continuous\continuous_query_handle_impl.cpp" />
     <ClCompile Include="..\..\src\impl\cache\query\query_impl.cpp" />
     <ClCompile Include="..\..\src\impl\ignite_environment.cpp" />
     <ClCompile Include="..\..\src\impl\ignite_impl.cpp" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/project/vs/core.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/project/vs/core.vcxproj.filters b/modules/platforms/cpp/core/project/vs/core.vcxproj.filters
index c5fb532..9cb5f78 100644
--- a/modules/platforms/cpp/core/project/vs/core.vcxproj.filters
+++ b/modules/platforms/cpp/core/project/vs/core.vcxproj.filters
@@ -46,6 +46,9 @@
     <ClCompile Include="..\..\src\impl\cache\query\query_batch.cpp">
       <Filter>Code\impl\cache\query</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\impl\cache\query\continuous\continuous_query_handle_impl.cpp">
+      <Filter>Code\impl\cache\query\continuous</Filter>
+    </ClCompile>
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\..\include\ignite\impl\cache\cache_impl.h">
@@ -144,6 +147,24 @@
     <ClInclude Include="..\..\include\ignite\impl\cache\query\query_batch.h">
       <Filter>Code\impl\cache\query</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\include\ignite\cache\query\continuous\continuous_query_handle.h">
+      <Filter>Code\cache\query\continuous</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\cache\query\continuous\continuous_query.h">
+      <Filter>Code\cache\query\continuous</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\impl\cache\query\continuous\continuous_query_handle_impl.h">
+      <Filter>Code\impl\cache\query\continuous</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\cache\event\cache_entry_event_listener.h">
+      <Filter>Code\cache\event</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\cache\event\cache_entry_event.h">
+      <Filter>Code\cache\event</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\impl\cache\query\continuous\continuous_query_impl.h">
+      <Filter>Code\impl\cache\query\continuous</Filter>
+    </ClInclude>
   </ItemGroup>
   <ItemGroup>
     <Filter Include="Code">
@@ -176,5 +197,14 @@
     <Filter Include="Code\transactions">
       <UniqueIdentifier>{146fe661-0ad3-4d51-83a3-fce8a897e84d}</UniqueIdentifier>
     </Filter>
+    <Filter Include="Code\cache\query\continuous">
+      <UniqueIdentifier>{2056dfc8-4ced-4658-b2b7-a8c81a7ef797}</UniqueIdentifier>
+    </Filter>
+    <Filter Include="Code\impl\cache\query\continuous">
+      <UniqueIdentifier>{d633f819-7b30-4e26-81ec-f708d1c1ff8e}</UniqueIdentifier>
+    </Filter>
+    <Filter Include="Code\cache\event">
+      <UniqueIdentifier>{e03c3690-ff22-4c78-83a0-b77cebb7f980}</UniqueIdentifier>
+    </Filter>
   </ItemGroup>
 </Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp b/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp
index 0630921..ad69d45 100644
--- a/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp
+++ b/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp
@@ -26,9 +26,11 @@ using namespace ignite::java;
 using namespace ignite::common;
 using namespace ignite::cache;
 using namespace ignite::cache::query;
+using namespace ignite::cache::query::continuous;
 using namespace ignite::impl;
 using namespace ignite::impl::binary;
 using namespace ignite::impl::cache::query;
+using namespace ignite::impl::cache::query::continuous;
 using namespace ignite::impl::interop;
 using namespace ignite::binary;
 
@@ -89,6 +91,9 @@ namespace ignite
             /** Operation: PutIfAbsent. */
             const int32_t OP_PUT_IF_ABSENT = 28;
 
+            /** Operation: CONTINUOUS query. */
+            const int32_t OP_QRY_CONTINUOUS = 29;
+
             /** Operation: SCAN query. */
             const int32_t OP_QRY_SCAN = 30;
 
@@ -301,6 +306,32 @@ namespace ignite
             {
                 return QueryInternal(qry, OP_QRY_SQL_FIELDS, err);
             }
+
+            ContinuousQueryHandleImpl* CacheImpl::QueryContinuous(const SharedPointer<ContinuousQueryImplBase> qry,
+                const SqlQuery& initialQry, IgniteError& err)
+            {
+                return QueryContinuous(qry, initialQry, OP_QRY_SQL, OP_QRY_CONTINUOUS, err);
+            }
+
+            ContinuousQueryHandleImpl* CacheImpl::QueryContinuous(const SharedPointer<ContinuousQueryImplBase> qry,
+                const TextQuery& initialQry, IgniteError& err)
+            {
+                return QueryContinuous(qry, initialQry, OP_QRY_TEXT, OP_QRY_CONTINUOUS, err);
+            }
+
+            ContinuousQueryHandleImpl* CacheImpl::QueryContinuous(const SharedPointer<ContinuousQueryImplBase> qry,
+                const ScanQuery& initialQry, IgniteError& err)
+            {
+                return QueryContinuous(qry, initialQry, OP_QRY_SCAN, OP_QRY_CONTINUOUS, err);
+            }
+
+            ContinuousQueryHandleImpl* CacheImpl::QueryContinuous(const SharedPointer<ContinuousQueryImplBase> qry,
+                IgniteError& err)
+            {
+                struct { void Write(BinaryRawWriter&) const { }} dummy;
+
+                return QueryContinuous(qry, dummy, -1, OP_QRY_CONTINUOUS, err);
+            }
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/src/impl/cache/query/continuous/continuous_query_handle_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/cache/query/continuous/continuous_query_handle_impl.cpp b/modules/platforms/cpp/core/src/impl/cache/query/continuous/continuous_query_handle_impl.cpp
new file mode 100644
index 0000000..04e64c9
--- /dev/null
+++ b/modules/platforms/cpp/core/src/impl/cache/query/continuous/continuous_query_handle_impl.cpp
@@ -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.
+ */
+
+#include "ignite/impl/cache/query/continuous/continuous_query_handle_impl.h"
+
+using namespace ignite::common::concurrent;
+using namespace ignite::jni::java;
+using namespace ignite::java;
+using namespace ignite::impl::interop;
+using namespace ignite::impl::binary;
+
+namespace ignite
+{
+    namespace impl
+    {
+        namespace cache
+        {
+            namespace query
+            {
+                namespace continuous
+                {
+                    enum Command
+                    {
+                        GET_INITIAL_QUERY = 0,
+
+                        CLOSE = 1
+                    };
+
+                    ContinuousQueryHandleImpl::ContinuousQueryHandleImpl(SP_IgniteEnvironment env, int64_t handle, jobject javaRef) :
+                        env(env),
+                        handle(handle),
+                        javaRef(javaRef),
+                        mutex(),
+                        extracted(false)
+                    {
+                        // No-op.
+                    }
+
+                    ContinuousQueryHandleImpl::~ContinuousQueryHandleImpl()
+                    {
+                        env.Get()->Context()->TargetInLongOutLong(javaRef, CLOSE, 0);
+
+                        JniContext::Release(javaRef);
+
+                        env.Get()->GetHandleRegistry().Release(handle);
+                    }
+
+                    QueryCursorImpl* ContinuousQueryHandleImpl::GetInitialQueryCursor(IgniteError& err)
+                    {
+                        CsLockGuard guard(mutex);
+
+                        if (extracted)
+                        {
+                            err = IgniteError(IgniteError::IGNITE_ERR_GENERIC,
+                                "GetInitialQueryCursor() can be called only once.");
+
+                            return 0;
+                        }
+
+                        JniErrorInfo jniErr;
+
+                        jobject res = env.Get()->Context()->TargetOutObject(javaRef, GET_INITIAL_QUERY, &jniErr);
+
+                        IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, &err);
+
+                        if (jniErr.code != IGNITE_JNI_ERR_SUCCESS)
+                            return 0;
+
+                        extracted = true;
+
+                        return new QueryCursorImpl(env, res);
+                    }
+
+                    void ContinuousQueryHandleImpl::SetQuery(SP_ContinuousQueryImplBase query)
+                    {
+                        qry = query;
+                    }
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/src/impl/handle_registry.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/handle_registry.cpp b/modules/platforms/cpp/core/src/impl/handle_registry.cpp
index c447faa..069e996 100644
--- a/modules/platforms/cpp/core/src/impl/handle_registry.cpp
+++ b/modules/platforms/cpp/core/src/impl/handle_registry.cpp
@@ -23,83 +23,67 @@ namespace ignite
 {
     namespace impl
     {
-        HandleRegistryEntry::~HandleRegistryEntry()
-        {
-            // No-op.
-        }
-
         HandleRegistrySegment::HandleRegistrySegment() : 
-            map(new std::map<int64_t, SharedPointer<HandleRegistryEntry>>()), mux(new CriticalSection())
+            map(),
+            mux()
         {
             // No-op.
         }
 
         HandleRegistrySegment::~HandleRegistrySegment()
         {
-            delete map;
-            delete mux;
+            // No-op.
         }
 
-        SharedPointer<HandleRegistryEntry> HandleRegistrySegment::Get(int64_t hnd)
+        SharedPointer<void> HandleRegistrySegment::Get(int64_t hnd)
         {
-            mux->Enter();
+            typedef std::map<int64_t, SharedPointer<void>> Map;
 
-            SharedPointer<HandleRegistryEntry> res = (*map)[hnd];
+            CsLockGuard guard(mux);
 
-            mux->Leave();
+            Map::const_iterator it = map.find(hnd);
+            if (it == map.end())
+                return SharedPointer<void>();
 
-            return res;
+            return it->second;
         }
 
-        void HandleRegistrySegment::Put(int64_t hnd, const SharedPointer<HandleRegistryEntry>& entry)
+        void HandleRegistrySegment::Put(int64_t hnd, const SharedPointer<void>& entry)
         {
-            mux->Enter();
-
-            (*map)[hnd] = entry;
+            CsLockGuard guard(mux);
 
-            mux->Leave();
+            map[hnd] = entry;
         }
 
         void HandleRegistrySegment::Remove(int64_t hnd)
         {
-            mux->Enter();
+            CsLockGuard guard(mux);
 
-            map->erase(hnd);
-
-            mux->Leave();
+            map.erase(hnd);
         }
 
         void HandleRegistrySegment::Clear()
         {
-            mux->Enter();
-
-            map->erase(map->begin(), map->end());
+            CsLockGuard guard(mux);
 
-            mux->Leave();
+            map.clear();
         }
 
-        HandleRegistry::HandleRegistry(int32_t fastCap, int32_t slowSegmentCnt)
+        HandleRegistry::HandleRegistry(int32_t fastCap, int32_t slowSegmentCnt) :
+            fastCap(fastCap),
+            fastCtr(0),
+            fast(new SharedPointer<void>[fastCap]),
+            slowSegmentCnt(slowSegmentCnt),
+            slowCtr(fastCap),
+            slow(new HandleRegistrySegment*[slowSegmentCnt]),
+            closed(0)
         {
-            this->fastCap = fastCap;
+            for (int32_t i = 0; i < fastCap; i++)
+                fast[i] = SharedPointer<void>();
 
-            fastCtr = 0;
-
-            fast = new SharedPointer<HandleRegistryEntry>[fastCap];
-
-            for (int i = 0; i < fastCap; i++)
-                fast[i] = SharedPointer<HandleRegistryEntry>();
-
-            this->slowSegmentCnt = slowSegmentCnt;
-
-            slowCtr = fastCap;
-
-            slow = new HandleRegistrySegment*[slowSegmentCnt];
-
-            for (int i = 0; i < slowSegmentCnt; i++)
+            for (int32_t i = 0; i < slowSegmentCnt; i++)
                 slow[i] = new HandleRegistrySegment();
 
-            closed = 0;
-
             Memory::Fence();
         }
 
@@ -115,22 +99,22 @@ namespace ignite
             delete[] slow;
         }
 
-        int64_t HandleRegistry::Allocate(const SharedPointer<HandleRegistryEntry>& target)
+        int64_t HandleRegistry::Allocate(const SharedPointer<void>& target)
         {
             return Allocate0(target, false, false);
         }
 
-        int64_t HandleRegistry::AllocateCritical(const SharedPointer<HandleRegistryEntry>& target)
+        int64_t HandleRegistry::AllocateCritical(const SharedPointer<void>& target)
         {
             return Allocate0(target, true, false);
         }
 
-        int64_t HandleRegistry::AllocateSafe(const SharedPointer<HandleRegistryEntry>& target)
+        int64_t HandleRegistry::AllocateSafe(const SharedPointer<void>& target)
         {
             return Allocate0(target, false, true);
         }
 
-        int64_t HandleRegistry::AllocateCriticalSafe(const SharedPointer<HandleRegistryEntry>& target)
+        int64_t HandleRegistry::AllocateCriticalSafe(const SharedPointer<void>& target)
         {
             return Allocate0(target, true, true);
         }
@@ -138,10 +122,10 @@ namespace ignite
         void HandleRegistry::Release(int64_t hnd)
         {
             if (hnd < fastCap)
-                fast[static_cast<int32_t>(hnd)] = SharedPointer<HandleRegistryEntry>();
+                fast[static_cast<int32_t>(hnd)] = SharedPointer<void>();
             else
             {
-                HandleRegistrySegment* segment = *(slow + hnd % slowSegmentCnt);
+                HandleRegistrySegment* segment = slow[hnd % slowSegmentCnt];
 
                 segment->Remove(hnd);
             }
@@ -149,7 +133,7 @@ namespace ignite
             Memory::Fence();
         }
 
-        SharedPointer<HandleRegistryEntry> HandleRegistry::Get(int64_t hnd)
+        SharedPointer<void> HandleRegistry::Get(int64_t hnd)
         {
             Memory::Fence();
 
@@ -157,7 +141,7 @@ namespace ignite
                 return fast[static_cast<int32_t>(hnd)];
             else
             {
-                HandleRegistrySegment* segment = *(slow + hnd % slowSegmentCnt);
+                HandleRegistrySegment* segment = slow[hnd % slowSegmentCnt];
 
                 return segment->Get(hnd);
             }
@@ -168,16 +152,16 @@ namespace ignite
             if (Atomics::CompareAndSet32(&closed, 0, 1))
             {
                 // Cleanup fast-path handles.
-                for (int i = 0; i < fastCap; i++)
-                    fast[i] = SharedPointer<HandleRegistryEntry>();
+                for (int32_t i = 0; i < fastCap; i++)
+                    fast[i] = SharedPointer<void>();
 
                 // Cleanup slow-path handles.
-                for (int i = 0; i < slowSegmentCnt; i++)
-                    (*(slow + i))->Clear();
+                for (int32_t i = 0; i < slowSegmentCnt; i++)
+                    slow[i]->Clear();
             }
         }
 
-        int64_t HandleRegistry::Allocate0(const SharedPointer<HandleRegistryEntry>& target, bool critical, bool safe)
+        int64_t HandleRegistry::Allocate0(const SharedPointer<void>& target, bool critical, bool safe)
         {
             // Check closed state.
             Memory::Fence();
@@ -201,7 +185,7 @@ namespace ignite
 
                         if (safe && closed == 1)
                         {
-                            fast[fastIdx] = SharedPointer<HandleRegistryEntry>();
+                            fast[fastIdx] = SharedPointer<void>();
 
                             return -1;
                         }
@@ -214,7 +198,7 @@ namespace ignite
             // Either allocating on slow-path, or fast-path can no longer accomodate more entries.
             int64_t slowIdx = Atomics::IncrementAndGet64(&slowCtr) - 1;
 
-            HandleRegistrySegment* segment = *(slow + slowIdx % slowSegmentCnt);
+            HandleRegistrySegment* segment = slow[slowIdx % slowSegmentCnt];
 
             segment->Put(slowIdx, target);
 


[10/50] [abbrv] ignite git commit: IGNITE-1443: Implemented ContinuousQuery for C++

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/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 4947b94..b058f7c 100644
--- a/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
+++ b/modules/platforms/cpp/core/src/impl/ignite_environment.cpp
@@ -18,6 +18,7 @@
 #include "ignite/impl/interop/interop_external_memory.h"
 #include "ignite/impl/binary/binary_reader_impl.h"
 #include "ignite/impl/ignite_environment.h"
+#include "ignite/cache/query/continuous/continuous_query.h"
 #include "ignite/binary/binary.h"
 #include "ignite/impl/binary/binary_type_updater_impl.h"
 
@@ -26,6 +27,7 @@ using namespace ignite::jni::java;
 using namespace ignite::impl::interop;
 using namespace ignite::impl::binary;
 using namespace ignite::binary;
+using namespace ignite::impl::cache::query::continuous;
 
 namespace ignite 
 {
@@ -36,6 +38,8 @@ namespace ignite
         */
         enum CallbackOp
         {
+            CONTINUOUS_QUERY_LISTENER_APPLY = 18,
+            CONTINUOUS_QUERY_FILTER_RELEASE = 21,
             REALLOC = 36,
             ON_START = 49,
             ON_STOP = 50 
@@ -50,11 +54,36 @@ namespace ignite
          */
         long long IGNITE_CALL InLongOutLong(void* target, int type, long long val)
         {
-            if (type == ON_STOP)
+            SharedPointer<IgniteEnvironment>* env = static_cast<SharedPointer<IgniteEnvironment>*>(target);
+
+            switch (type)
             {
-                SharedPointer<IgniteEnvironment>* ptr = static_cast<SharedPointer<IgniteEnvironment>*>(target);
+                case ON_STOP:
+                {
+                    delete env;
+
+                    break;
+                }
+
+                case CONTINUOUS_QUERY_LISTENER_APPLY:
+                {
+                    SharedPointer<InteropMemory> mem = env->Get()->GetMemory(val);
+
+                    env->Get()->OnContinuousQueryListenerApply(mem);
+
+                    break;
+                }
+
+                case CONTINUOUS_QUERY_FILTER_RELEASE:
+                {
+                    // No-op.
+                    break;
+                }
 
-                delete ptr;
+                default:
+                {
+                    break;
+                }
             }
 
             return 0;
@@ -73,26 +102,43 @@ namespace ignite
         long long IGNITE_CALL InLongLongLongObjectOutLong(void* target, int type, long long val1, long long val2, 
             long long val3, void* arg)
         {
-            if (type == ON_START)
-            {
-                SharedPointer<IgniteEnvironment>* ptr = static_cast<SharedPointer<IgniteEnvironment>*>(target);
+            SharedPointer<IgniteEnvironment>* env = static_cast<SharedPointer<IgniteEnvironment>*>(target);
 
-                ptr->Get()->OnStartCallback(val1, reinterpret_cast<jobject>(arg));
-            }
-            else if (type == REALLOC)
+            switch (type)
             {
-                SharedPointer<IgniteEnvironment>* env = static_cast<SharedPointer<IgniteEnvironment>*>(target);
+                case ON_START:
+                {
+                    env->Get()->OnStartCallback(val1, reinterpret_cast<jobject>(arg));
+
+                    break;
+                }
 
-                SharedPointer<InteropMemory> mem = env->Get()->GetMemory(val1);
+                case REALLOC:
+                {
+                    SharedPointer<InteropMemory> mem = env->Get()->GetMemory(val1);
 
-                mem.Get()->Reallocate(static_cast<int32_t>(val2));
+                    mem.Get()->Reallocate(static_cast<int32_t>(val2));
+
+                    break;
+                }
+
+                default:
+                {
+                    break;
+                }
             }
 
             return 0;
         }
 
-        IgniteEnvironment::IgniteEnvironment() : ctx(SharedPointer<JniContext>()), latch(new SingleLatch), name(0),
-            proc(), metaMgr(new BinaryTypeManager()), metaUpdater(0)
+        IgniteEnvironment::IgniteEnvironment() :
+            ctx(SharedPointer<JniContext>()),
+            latch(new SingleLatch),
+            name(0),
+            proc(),
+            metaMgr(new BinaryTypeManager()),
+            metaUpdater(0),
+            registry(DEFAULT_FAST_PATH_CONTAINERS_CAP, DEFAULT_SLOW_PATH_CONTAINERS_CAP)
         {
             // No-op.
         }
@@ -107,7 +153,7 @@ namespace ignite
 
         JniHandlers IgniteEnvironment::GetJniHandlers(SharedPointer<IgniteEnvironment>* target)
         {
-            JniHandlers hnds = JniHandlers();
+            JniHandlers hnds;
 
             hnds.target = target;
 
@@ -193,6 +239,11 @@ namespace ignite
                 ctx.Get()->ProcessorReleaseStart(proc.Get());
         }
 
+        HandleRegistry& IgniteEnvironment::GetHandleRegistry()
+        {
+            return registry;
+        }
+
         void IgniteEnvironment::OnStartCallback(long long memPtr, jobject proc)
         {
             this->proc = jni::JavaGlobalRef(*ctx.Get(), proc);
@@ -212,6 +263,23 @@ namespace ignite
             else
                 name = 0;
         }
+
+        void IgniteEnvironment::OnContinuousQueryListenerApply(SharedPointer<InteropMemory>& mem)
+        {
+            InteropInputStream stream(mem.Get());
+            BinaryReaderImpl reader(&stream);
+
+            int64_t qryHandle = reader.ReadInt64();
+
+            ContinuousQueryImplBase* contQry = reinterpret_cast<ContinuousQueryImplBase*>(registry.Get(qryHandle).Get());
+
+            if (contQry)
+            {
+                BinaryRawReader rawReader(&reader);
+
+                contQry->ReadAndProcessEvents(rawReader);
+            }
+        }
     }
 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/examples/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/Makefile.am b/modules/platforms/cpp/examples/Makefile.am
index cda5132..e1c7905 100644
--- a/modules/platforms/cpp/examples/Makefile.am
+++ b/modules/platforms/cpp/examples/Makefile.am
@@ -21,5 +21,6 @@ SUBDIRS = \
     putget-example \
     odbc-example \
     query-example \
+	continuous-query-example \
     include
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/examples/configure.ac
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/configure.ac b/modules/platforms/cpp/examples/configure.ac
index 6ddd7c8..d167cf0 100644
--- a/modules/platforms/cpp/examples/configure.ac
+++ b/modules/platforms/cpp/examples/configure.ac
@@ -56,6 +56,7 @@ AC_CONFIG_FILES([ \
     putget-example/Makefile \
     odbc-example/Makefile \
     query-example/Makefile \
+    continuous-query-example/Makefile \
 ])
 
 AC_OUTPUT

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/examples/continuous-query-example/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/continuous-query-example/Makefile.am b/modules/platforms/cpp/examples/continuous-query-example/Makefile.am
new file mode 100644
index 0000000..2566689
--- /dev/null
+++ b/modules/platforms/cpp/examples/continuous-query-example/Makefile.am
@@ -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.
+##
+
+ACLOCAL_AMFLAGS =-I m4
+
+noinst_PROGRAMS = ignite-continuous-query-example
+
+AM_CPPFLAGS = \
+    -I@top_srcdir@/include \
+    -I@top_srcdir@/../core/include \
+    -I@top_srcdir@/../core/os/linux/include \
+    -I@top_srcdir@/../common/include \
+    -I@top_srcdir@/../common/os/linux/include \
+    -I@top_srcdir@/../binary/include \
+    -I@top_srcdir@/../jni/include \
+    -I@top_srcdir@/../jni/os/linux/include \
+    -I$(JAVA_HOME)/include \
+    -I$(JAVA_HOME)/include/linux \
+    -DIGNITE_IMPL \
+    -D__STDC_LIMIT_MACROS \
+    -D__STDC_CONSTANT_MACROS
+
+AM_CXXFLAGS = \
+    -Wall \
+    -std=c++0x
+
+ignite_continuous_query_example_LDADD = \
+    @top_srcdir@/../core/libignite.la \
+    -lpthread
+
+ignite_continuous_query_example_LDFLAGS = \
+    -static-libtool-libs
+
+ignite_continuous_query_example_SOURCES = \
+    src/continuous_query_example.cpp
+
+run-check: check
+	./ignite-continuous-query-example -p
+
+clean-local: clean-check
+	$(RM) *.gcno *.gcda
+
+clean-check:
+	$(RM) $(ignite_continuous_query_example_OBJECTS)

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/examples/continuous-query-example/config/continuous-query-example.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/continuous-query-example/config/continuous-query-example.xml b/modules/platforms/cpp/examples/continuous-query-example/config/continuous-query-example.xml
new file mode 100644
index 0000000..bdc1e92
--- /dev/null
+++ b/modules/platforms/cpp/examples/continuous-query-example/config/continuous-query-example.xml
@@ -0,0 +1,52 @@
+<?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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+
+        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <!--
+                        Ignite provides several options for automatic discovery that can be used
+                        instead os static IP based discovery.
+                    -->
+                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
+                    <!--<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">-->
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47550..47551</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/examples/continuous-query-example/project/vs/continuous-query-example.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/continuous-query-example/project/vs/continuous-query-example.vcxproj b/modules/platforms/cpp/examples/continuous-query-example/project/vs/continuous-query-example.vcxproj
new file mode 100644
index 0000000..dfc5f9d
--- /dev/null
+++ b/modules/platforms/cpp/examples/continuous-query-example/project/vs/continuous-query-example.vcxproj
@@ -0,0 +1,110 @@
+\ufeff<?xml version="1.0" encoding="utf-8"?>
+<Project DefaultTargets="Build" ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
+  <ItemGroup Label="ProjectConfigurations">
+    <ProjectConfiguration Include="Release|Win32">
+      <Configuration>Release</Configuration>
+      <Platform>Win32</Platform>
+    </ProjectConfiguration>
+    <ProjectConfiguration Include="Release|x64">
+      <Configuration>Release</Configuration>
+      <Platform>x64</Platform>
+    </ProjectConfiguration>
+  </ItemGroup>
+  <PropertyGroup Label="Globals">
+    <RootNamespace>igniteexamples</RootNamespace>
+    <ProjectGuid>{73BB124A-0CD4-4961-A6CD-61F9C71028A6}</ProjectGuid>
+  </PropertyGroup>
+  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.Default.props" />
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="Configuration">
+    <ConfigurationType>Application</ConfigurationType>
+    <UseDebugLibraries>false</UseDebugLibraries>
+    <PlatformToolset>v100</PlatformToolset>
+    <WholeProgramOptimization>true</WholeProgramOptimization>
+    <CharacterSet>Unicode</CharacterSet>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="Configuration">
+    <ConfigurationType>Application</ConfigurationType>
+    <UseDebugLibraries>false</UseDebugLibraries>
+    <PlatformToolset>v100</PlatformToolset>
+    <WholeProgramOptimization>true</WholeProgramOptimization>
+    <CharacterSet>Unicode</CharacterSet>
+  </PropertyGroup>
+  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.props" />
+  <ImportGroup Label="ExtensionSettings">
+  </ImportGroup>
+  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'" Label="PropertySheets">
+    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
+  </ImportGroup>
+  <ImportGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" Label="PropertySheets">
+    <Import Project="$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(UserRootDir)\Microsoft.Cpp.$(Platform).user.props')" Label="LocalAppDataPlatform" />
+  </ImportGroup>
+  <PropertyGroup Label="UserMacros" />
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
+    <LinkIncremental>false</LinkIncremental>
+  </PropertyGroup>
+  <PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
+    <LinkIncremental>false</LinkIncremental>
+  </PropertyGroup>
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
+    <ClCompile>
+      <WarningLevel>Level3</WarningLevel>
+      <PrecompiledHeader>
+      </PrecompiledHeader>
+      <Optimization>MaxSpeed</Optimization>
+      <FunctionLevelLinking>true</FunctionLevelLinking>
+      <IntrinsicFunctions>true</IntrinsicFunctions>
+      <PreprocessorDefinitions>WIN32;NDEBUG;_CONSOLE;_LIB;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <AdditionalIncludeDirectories>$(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;..\..\..\include;..\..\..\..\jni\os\win\include;..\..\..\..\jni\include;..\..\..\..\common\os\win\include;..\..\..\..\common\include;..\..\..\..\binary\include;..\..\..\..\core\os\win\include;..\..\..\..\core\include;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
+    </ClCompile>
+    <Link>
+      <SubSystem>Console</SubSystem>
+      <GenerateDebugInformation>true</GenerateDebugInformation>
+      <EnableCOMDATFolding>true</EnableCOMDATFolding>
+      <OptimizeReferences>true</OptimizeReferences>
+      <AdditionalDependencies>jvm.lib;ignite.jni.lib;ignite.binary.lib;ignite.core.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalLibraryDirectories>..\..\..\..\project\vs\$(Platform)\$(Configuration)\;$(JAVA_HOME)\lib;%(AdditionalLibraryDirectories)</AdditionalLibraryDirectories>
+    </Link>
+    <PostBuildEvent>
+      <Command>copy "$(ProjectDir)..\..\..\..\project\vs\$(Platform)\$(Configuration)\ignite.common.dll" "$(OutDir)"
+copy "$(ProjectDir)..\..\..\..\project\vs\$(Platform)\$(Configuration)\ignite.core.dll" "$(OutDir)"</Command>
+    </PostBuildEvent>
+  </ItemDefinitionGroup>
+  <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
+    <ClCompile>
+      <WarningLevel>Level3</WarningLevel>
+      <PrecompiledHeader>
+      </PrecompiledHeader>
+      <Optimization>MaxSpeed</Optimization>
+      <FunctionLevelLinking>true</FunctionLevelLinking>
+      <IntrinsicFunctions>true</IntrinsicFunctions>
+      <PreprocessorDefinitions>WIN32;NDEBUG;_CONSOLE;_LIB;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <AdditionalIncludeDirectories>$(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;..\..\..\include;..\..\..\..\jni\os\win\include;..\..\..\..\jni\include;..\..\..\..\common\os\win\include;..\..\..\..\common\include;..\..\..\..\binary\include;..\..\..\..\core\os\win\include;..\..\..\..\core\include;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
+    </ClCompile>
+    <Link>
+      <SubSystem>Console</SubSystem>
+      <GenerateDebugInformation>true</GenerateDebugInformation>
+      <EnableCOMDATFolding>true</EnableCOMDATFolding>
+      <OptimizeReferences>true</OptimizeReferences>
+      <AdditionalDependencies>jvm.lib;ignite.jni.lib;ignite.binary.lib;ignite.core.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalLibraryDirectories>..\..\..\..\project\vs\$(Platform)\$(Configuration)\;$(JAVA_HOME)\lib;%(AdditionalLibraryDirectories)</AdditionalLibraryDirectories>
+    </Link>
+    <PostBuildEvent>
+      <Command>copy "$(ProjectDir)..\..\..\..\project\vs\$(Platform)\$(Configuration)\ignite.common.dll" "$(OutDir)"
+copy "$(ProjectDir)..\..\..\..\project\vs\$(Platform)\$(Configuration)\ignite.core.dll" "$(OutDir)"</Command>
+    </PostBuildEvent>
+  </ItemDefinitionGroup>
+  <ItemGroup>
+    <ClCompile Include="..\..\src\continuous_query_example.cpp" />
+  </ItemGroup>
+  <ItemGroup>
+    <ClInclude Include="..\..\..\include\ignite\examples\address.h" />
+    <ClInclude Include="..\..\..\include\ignite\examples\organization.h" />
+    <ClInclude Include="..\..\..\include\ignite\examples\person.h" />
+  </ItemGroup>
+  <ItemGroup>
+    <None Include="..\..\config\continuous-query-example.xml" />
+  </ItemGroup>
+  <Import Project="$(VCTargetsPath)\Microsoft.Cpp.targets" />
+  <ImportGroup Label="ExtensionTargets">
+  </ImportGroup>
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/examples/continuous-query-example/project/vs/continuous-query-example.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/continuous-query-example/project/vs/continuous-query-example.vcxproj.filters b/modules/platforms/cpp/examples/continuous-query-example/project/vs/continuous-query-example.vcxproj.filters
new file mode 100644
index 0000000..cf3bca9
--- /dev/null
+++ b/modules/platforms/cpp/examples/continuous-query-example/project/vs/continuous-query-example.vcxproj.filters
@@ -0,0 +1,35 @@
+\ufeff<?xml version="1.0" encoding="utf-8"?>
+<Project ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
+  <ItemGroup>
+    <ClInclude Include="..\..\..\include\ignite\examples\address.h">
+      <Filter>Header Files</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\..\include\ignite\examples\organization.h">
+      <Filter>Header Files</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\..\include\ignite\examples\person.h">
+      <Filter>Header Files</Filter>
+    </ClInclude>
+  </ItemGroup>
+  <ItemGroup>
+    <Filter Include="Source Files">
+      <UniqueIdentifier>{35cb32b7-bf2e-440f-9b32-80d392d81847}</UniqueIdentifier>
+    </Filter>
+    <Filter Include="Header Files">
+      <UniqueIdentifier>{b355095f-b4e2-4324-9516-854828c876ff}</UniqueIdentifier>
+    </Filter>
+    <Filter Include="Config">
+      <UniqueIdentifier>{3799efd0-3cfe-47e2-9e9e-a51b25bf40ef}</UniqueIdentifier>
+    </Filter>
+  </ItemGroup>
+  <ItemGroup>
+    <ClCompile Include="..\..\src\continuous_query_example.cpp">
+      <Filter>Source Files</Filter>
+    </ClCompile>
+  </ItemGroup>
+  <ItemGroup>
+    <None Include="..\..\config\continuous-query-example.xml">
+      <Filter>Config</Filter>
+    </None>
+  </ItemGroup>
+</Project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/examples/continuous-query-example/src/continuous_query_example.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/continuous-query-example/src/continuous_query_example.cpp b/modules/platforms/cpp/examples/continuous-query-example/src/continuous_query_example.cpp
new file mode 100644
index 0000000..b08d4b8
--- /dev/null
+++ b/modules/platforms/cpp/examples/continuous-query-example/src/continuous_query_example.cpp
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <stdint.h>
+#include <iostream>
+
+#include "ignite/ignition.h"
+#include "ignite/cache/query/continuous/continuous_query.h"
+
+#include "ignite/examples/organization.h"
+#include "ignite/examples/person.h"
+
+using namespace ignite;
+using namespace cache;
+using namespace query;
+
+using namespace examples;
+
+/** Cache name. */
+const char* CACHE_NAME = "cpp_cache_continuous_query";
+
+/*
+ * Listener class.
+ */
+template<typename K, typename V>
+class Listener : public event::CacheEntryEventListener<K, V>
+{
+public:
+    /*
+     * Default constructor.
+     */
+    Listener()
+    {
+        // No-op.
+    }
+
+    /**
+     * Event callback.
+     *
+     * @param evts Events.
+     * @param num Events number.
+     */
+    virtual void OnEvent(const CacheEntryEvent<K, V>* evts, uint32_t num)
+    {
+        for (uint32_t i = 0; i < num; ++i)
+        {
+            std::cout << "Queried entry [key=" << (evts[i].HasValue() ? evts[i].GetKey() : K())
+                      << ", val=" << (evts[i].HasValue() ? evts[i].GetValue() : V()) << ']'
+                      << std::endl;
+        }
+    }
+};
+
+int main()
+{
+    IgniteConfiguration cfg;
+
+    cfg.springCfgPath = "platforms/cpp/examples/continuous-query-example/config/continuous-query-example.xml";
+
+    try
+    {
+        // Start a node.
+        Ignite ignite = Ignition::Start(cfg);
+
+        std::cout << std::endl;
+        std::cout << ">>> Cache continuous query example started." << std::endl;
+        std::cout << std::endl;
+
+        // Get cache instance.
+        Cache<int32_t, std::string> cache = ignite.GetOrCreateCache<int32_t, std::string>(CACHE_NAME);
+
+        cache.Clear();
+
+        const int32_t keyCnt = 20;
+
+        for (int32_t i = 0; i < keyCnt; ++i)
+        {
+            std::stringstream builder;
+
+            builder << i;
+
+            cache.Put(i, builder.str());
+        }
+
+        // Declaring listener.
+        Listener<int, std::string> listener;
+
+        // Declaring continuous query.
+        continuous::ContinuousQuery<int, std::string> qry(MakeReference(listener));
+
+        {
+            // Continous query scope. Query is closed when scope is left.
+            continuous::ContinuousQueryHandle<int, std::string> handle = cache.QueryContinuous(qry);
+
+            // Add a few more keys and watch more query notifications.
+            for (int32_t i = keyCnt; i < keyCnt + 5; ++i)
+            {
+                std::stringstream builder;
+
+                builder << i;
+
+                cache.Put(i, builder.str());
+            }
+
+            // Let user wait while callback is notified about remaining puts.
+            std::cout << std::endl;
+            std::cout << ">>> Press 'Enter' to continue..." << std::endl;
+            std::cout << std::endl;
+
+            std::cin.get();
+        }
+
+        // Stop node.
+        Ignition::StopAll(false);
+    }
+    catch (IgniteError& err)
+    {
+        std::cout << "An error occurred: " << err.GetText() << std::endl;
+    }
+
+    std::cout << std::endl;
+    std::cout << ">>> Example finished, press 'Enter' to exit ..." << std::endl;
+    std::cout << std::endl;
+
+    std::cin.get();
+
+    return 0;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/examples/include/ignite/examples/person.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/include/ignite/examples/person.h b/modules/platforms/cpp/examples/include/ignite/examples/person.h
index 2c92660..86c51c8 100644
--- a/modules/platforms/cpp/examples/include/ignite/examples/person.h
+++ b/modules/platforms/cpp/examples/include/ignite/examples/person.h
@@ -46,7 +46,7 @@ namespace ignite
                 // No-op.
             }
 
-            std::string ToString()
+            std::string ToString() const
             {
                 std::ostringstream oss;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/examples/project/vs/ignite-examples.sln
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/project/vs/ignite-examples.sln b/modules/platforms/cpp/examples/project/vs/ignite-examples.sln
index 89f609f..d839f09 100644
--- a/modules/platforms/cpp/examples/project/vs/ignite-examples.sln
+++ b/modules/platforms/cpp/examples/project/vs/ignite-examples.sln
@@ -7,6 +7,8 @@ Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "odbc-example", "..\..\odbc-
 EndProject
 Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "query-example", "..\..\query-example\project\vs\query-example.vcxproj", "{9FB34AB4-01DD-4C6F-99BF-681019D0E4DD}"
 EndProject
+Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "continuous-query-example", "..\..\continuous-query-example\project\vs\continuous-query-example.vcxproj", "{73BB124A-0CD4-4961-A6CD-61F9C71028A6}"
+EndProject
 Global
 	GlobalSection(SolutionConfigurationPlatforms) = preSolution
 		Release|x64 = Release|x64
@@ -25,6 +27,10 @@ Global
 		{9FB34AB4-01DD-4C6F-99BF-681019D0E4DD}.Release|x64.Build.0 = Release|x64
 		{9FB34AB4-01DD-4C6F-99BF-681019D0E4DD}.Release|x86.ActiveCfg = Release|Win32
 		{9FB34AB4-01DD-4C6F-99BF-681019D0E4DD}.Release|x86.Build.0 = Release|Win32
+		{73BB124A-0CD4-4961-A6CD-61F9C71028A6}.Release|x64.ActiveCfg = Release|x64
+		{73BB124A-0CD4-4961-A6CD-61F9C71028A6}.Release|x64.Build.0 = Release|x64
+		{73BB124A-0CD4-4961-A6CD-61F9C71028A6}.Release|x86.ActiveCfg = Release|Win32
+		{73BB124A-0CD4-4961-A6CD-61F9C71028A6}.Release|x86.Build.0 = Release|Win32
 	EndGlobalSection
 	GlobalSection(SolutionProperties) = preSolution
 		HideSolutionNode = FALSE

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/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 442cc10..6289d73 100644
--- a/modules/platforms/cpp/jni/include/ignite/jni/java.h
+++ b/modules/platforms/cpp/jni/include/ignite/jni/java.h
@@ -389,7 +389,7 @@ namespace ignite
                 void TargetListenFutureForOperation(jobject obj, long long futId, int typ, int opId);
 
                 jobject CacheOutOpQueryCursor(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL);
-                jobject CacheOutOpContinuousQuery(jobject obj, int type, long long memPtr);
+                jobject CacheOutOpContinuousQuery(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL);
 
                 jobject Acquire(jobject obj);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/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 698263d..e6f2f88 100644
--- a/modules/platforms/cpp/jni/src/java.cpp
+++ b/modules/platforms/cpp/jni/src/java.cpp
@@ -1357,13 +1357,13 @@ namespace ignite
                 return LocalToGlobal(env, res);
             }
 
-            jobject JniContext::CacheOutOpContinuousQuery(jobject obj, int type, long long memPtr) {
+            jobject JniContext::CacheOutOpContinuousQuery(jobject obj, int type, long long memPtr, JniErrorInfo* err) {
                 JNIEnv* env = Attach();
 
                 jobject res = env->CallObjectMethod(
                     obj, jvm->GetMembers().m_PlatformTarget_inStreamOutObject, type, memPtr);
 
-                ExceptionCheck(env);
+                ExceptionCheck(env, err);
 
                 return LocalToGlobal(env, res);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
index 753ae4c..6f85896 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
@@ -97,7 +97,7 @@
     </ClCompile>
     <Link>
       <GenerateDebugInformation>true</GenerateDebugInformation>
-      <AdditionalDependencies>$(JAVA_HOME)\lib\jvm.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_unit_test_framework-vc100-mt-gd-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalDependencies>$(JAVA_HOME)\lib\jvm.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_unit_test_framework-vc100-mt-gd-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
     </Link>
   </ItemDefinitionGroup>
   <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Debug|x64'">
@@ -130,7 +130,7 @@
       <GenerateDebugInformation>true</GenerateDebugInformation>
       <EnableCOMDATFolding>true</EnableCOMDATFolding>
       <OptimizeReferences>true</OptimizeReferences>
-      <AdditionalDependencies>$(JAVA_HOME)\lib\jvm.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_unit_test_framework-vc100-mt-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalDependencies>$(JAVA_HOME)\lib\jvm.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_unit_test_framework-vc100-mt-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
     </Link>
   </ItemDefinitionGroup>
   <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/project/vs/ignite.slnrel
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/project/vs/ignite.slnrel b/modules/platforms/cpp/project/vs/ignite.slnrel
index 1d874a8..6a37bf4 100644
--- a/modules/platforms/cpp/project/vs/ignite.slnrel
+++ b/modules/platforms/cpp/project/vs/ignite.slnrel
@@ -2,6 +2,9 @@
 Microsoft Visual Studio Solution File, Format Version 11.00
 # Visual Studio 2010
 Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "core", "..\..\core\project\vs\core.vcxproj", "{E2DEA693-F2EA-43C2-A813-053378F6E4DB}"
+	ProjectSection(ProjectDependencies) = postProject
+		{4F15669B-92EB-49F0-B774-8F19BAE0B960} = {4F15669B-92EB-49F0-B774-8F19BAE0B960}
+	EndProjectSection
 EndProject
 Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "ignite", "..\..\ignite\project\vs\ignite.vcxproj", "{69688B4D-3EE0-43F5-A1C6-29B5D2DDE949}"
 	ProjectSection(ProjectDependencies) = postProject

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/project/vs/ignite_x86.slnrel
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/project/vs/ignite_x86.slnrel b/modules/platforms/cpp/project/vs/ignite_x86.slnrel
index 4785579..abd0ad1 100644
--- a/modules/platforms/cpp/project/vs/ignite_x86.slnrel
+++ b/modules/platforms/cpp/project/vs/ignite_x86.slnrel
@@ -2,6 +2,9 @@
 Microsoft Visual Studio Solution File, Format Version 11.00
 # Visual Studio 2010
 Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "core", "..\..\core\project\vs\core.vcxproj", "{E2DEA693-F2EA-43C2-A813-053378F6E4DB}"
+	ProjectSection(ProjectDependencies) = postProject
+		{4F15669B-92EB-49F0-B774-8F19BAE0B960} = {4F15669B-92EB-49F0-B774-8F19BAE0B960}
+	EndProjectSection
 EndProject
 Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "ignite", "..\..\ignite\project\vs\ignite.vcxproj", "{69688B4D-3EE0-43F5-A1C6-29B5D2DDE949}"
 	ProjectSection(ProjectDependencies) = postProject


[31/50] [abbrv] ignite git commit: Web console beta-7.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js b/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js
index b123ab5..9590779 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js
@@ -158,7 +158,7 @@ const PREDEFINED_QUERIES = [
 ];
 
 // Var name generator function.
-const beenNameSeed = () => {
+const beanNameSeed = () => {
     let idx = '';
     const names = [];
 
@@ -174,1551 +174,1577 @@ const beenNameSeed = () => {
     };
 };
 
-export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator', (JavaTypes, eventGroups, generator) => {
-    class JavaTransformer extends AbstractTransformer {
-        static generator = generator;
-
-        // Mapping for objects to method call.
-        static METHOD_MAPPING = {
-            'org.apache.ignite.configuration.CacheConfiguration': {
-                id: (ccfg) => JavaTypes.toJavaName('cache', ccfg.findProperty('name').value),
-                args: '',
-                generator: (sb, id, ccfg) => {
-                    const cacheName = ccfg.findProperty('name').value;
-                    const dataSources = JavaTransformer.collectDataSources(ccfg);
-
-                    const javadoc = [
-                        `Create configuration for cache "${cacheName}".`,
-                        '',
-                        '@return Configured cache.'
-                    ];
+export default class IgniteJavaTransformer extends AbstractTransformer {
+    // Mapping for objects to method call.
+    static METHOD_MAPPING = {
+        'org.apache.ignite.configuration.CacheConfiguration': {
+            prefix: 'cache',
+            name: 'name',
+            args: '',
+            generator: (sb, id, ccfg) => {
+                const cacheName = ccfg.findProperty('name').value;
+                const dataSources = IgniteJavaTransformer.collectDataSources(ccfg);
+
+                const javadoc = [
+                    `Create configuration for cache "${cacheName}".`,
+                    '',
+                    '@return Configured cache.'
+                ];
 
-                    if (dataSources.length)
-                        javadoc.push('@throws Exception if failed to create cache configuration.');
+                if (dataSources.length)
+                    javadoc.push('@throws Exception if failed to create cache configuration.');
 
-                    JavaTransformer.commentBlock(sb, ...javadoc);
-                    sb.startBlock(`public static CacheConfiguration ${id}()${dataSources.length ? ' throws Exception' : ''} {`);
+                IgniteJavaTransformer.commentBlock(sb, ...javadoc);
+                sb.startBlock(`public static CacheConfiguration ${id}()${dataSources.length ? ' throws Exception' : ''} {`);
 
-                    JavaTransformer.constructBean(sb, ccfg, [], true);
+                IgniteJavaTransformer.constructBean(sb, ccfg, [], true);
 
-                    sb.emptyLine();
-                    sb.append(`return ${ccfg.id};`);
+                sb.emptyLine();
+                sb.append(`return ${ccfg.id};`);
 
-                    sb.endBlock('}');
+                sb.endBlock('}');
 
-                    return sb;
-                }
-            },
-            'org.apache.ignite.cache.store.jdbc.JdbcType': {
-                id: (type) => JavaTypes.toJavaName('jdbcType', JavaTypes.shortClassName(type.findProperty('valueType').value)),
-                args: 'ccfg.getName()',
-                generator: (sb, name, jdbcType) => {
-                    const javadoc = [
-                        `Create JDBC type for "${name}".`,
-                        '',
-                        '@param cacheName Cache name.',
-                        '@return Configured JDBC type.'
-                    ];
+                return sb;
+            }
+        },
+        'org.apache.ignite.cache.store.jdbc.JdbcType': {
+            prefix: 'jdbcType',
+            name: 'valueType',
+            args: 'ccfg.getName()',
+            generator: (sb, name, jdbcType) => {
+                const javadoc = [
+                    `Create JDBC type for "${name}".`,
+                    '',
+                    '@param cacheName Cache name.',
+                    '@return Configured JDBC type.'
+                ];
 
-                    JavaTransformer.commentBlock(sb, ...javadoc);
-                    sb.startBlock(`private static JdbcType ${name}(String cacheName) {`);
+                IgniteJavaTransformer.commentBlock(sb, ...javadoc);
+                sb.startBlock(`private static JdbcType ${name}(String cacheName) {`);
 
-                    const cacheName = jdbcType.findProperty('cacheName');
+                const cacheName = jdbcType.findProperty('cacheName');
 
-                    cacheName.clsName = 'var';
-                    cacheName.value = 'cacheName';
+                cacheName.clsName = 'var';
+                cacheName.value = 'cacheName';
 
-                    JavaTransformer.constructBean(sb, jdbcType);
+                IgniteJavaTransformer.constructBean(sb, jdbcType);
 
-                    sb.emptyLine();
-                    sb.append(`return ${jdbcType.id};`);
+                sb.emptyLine();
+                sb.append(`return ${jdbcType.id};`);
 
-                    sb.endBlock('}');
+                sb.endBlock('}');
 
-                    return sb;
-                }
+                return sb;
             }
-        };
-
-        // Append comment line.
-        static comment(sb, ...lines) {
-            _.forEach(lines, (line) => sb.append(`// ${line}`));
         }
+    };
 
-        // Append comment block.
-        static commentBlock(sb, ...lines) {
-            if (lines.length === 1)
-                sb.append(`/** ${_.head(lines)} **/`);
-            else {
-                sb.append('/**');
+    // Append comment line.
+    static comment(sb, ...lines) {
+        _.forEach(lines, (line) => sb.append(`// ${line}`));
+    }
 
-                _.forEach(lines, (line) => sb.append(` * ${line}`));
+    // Append comment block.
+    static commentBlock(sb, ...lines) {
+        if (lines.length === 1)
+            sb.append(`/** ${_.head(lines)} **/`);
+        else {
+            sb.append('/**');
 
-                sb.append(' **/');
-            }
+            _.forEach(lines, (line) => sb.append(` * ${line}`));
+
+            sb.append(' **/');
         }
+    }
 
-        /**
-         * @param {Bean} bean
-         */
-        static _newBean(bean) {
-            const shortClsName = JavaTypes.shortClassName(bean.clsName);
-
-            if (_.isEmpty(bean.arguments))
-                return `new ${shortClsName}()`;
-
-            const args = _.map(bean.arguments, (arg) => {
-                switch (arg.clsName) {
-                    case 'MAP':
-                        return arg.id;
-                    case 'BEAN':
-                        return this._newBean(arg.value);
-                    default:
-                        return this._toObject(arg.clsName, arg.value);
-                }
-            });
+    /**
+     * @param {Bean} bean
+     */
+    static _newBean(bean) {
+        const shortClsName = this.javaTypes.shortClassName(bean.clsName);
+
+        if (_.isEmpty(bean.arguments))
+            return `new ${shortClsName}()`;
+
+        const args = _.map(bean.arguments, (arg) => {
+            switch (arg.clsName) {
+                case 'MAP':
+                    return arg.id;
+                case 'BEAN':
+                    return this._newBean(arg.value);
+                default:
+                    return this._toObject(arg.clsName, arg.value);
+            }
+        });
 
-            if (bean.factoryMtd)
-                return `${shortClsName}.${bean.factoryMtd}(${args.join(', ')})`;
+        if (bean.factoryMtd)
+            return `${shortClsName}.${bean.factoryMtd}(${args.join(', ')})`;
 
-            return `new ${shortClsName}(${args.join(', ')})`;
-        }
+        return `new ${shortClsName}(${args.join(', ')})`;
+    }
 
-        /**
-         * @param {StringBuilder} sb
-         * @param {String} parentId
-         * @param {String} propertyName
-         * @param {String} value
-         * @private
-         */
-        static _setProperty(sb, parentId, propertyName, value) {
-            sb.append(`${parentId}.set${_.upperFirst(propertyName)}(${value});`);
-        }
+    /**
+     * @param {StringBuilder} sb
+     * @param {String} parentId
+     * @param {String} propertyName
+     * @param {String} value
+     * @private
+     */
+    static _setProperty(sb, parentId, propertyName, value) {
+        sb.append(`${parentId}.set${_.upperFirst(propertyName)}(${value});`);
+    }
 
-        /**
-         * @param {StringBuilder} sb
-         * @param {Array.<String>} vars
-         * @param {Boolean} limitLines
-         * @param {Bean} bean
-         * @param {String} id
-
-         * @private
-         */
-        static constructBean(sb, bean, vars = [], limitLines = false, id = bean.id) {
-            _.forEach(bean.arguments, (arg) => {
-                switch (arg.clsName) {
-                    case 'MAP':
-                        this._constructMap(sb, arg, vars);
+    /**
+     * @param {StringBuilder} sb
+     * @param {Array.<String>} vars
+     * @param {Boolean} limitLines
+     * @param {Bean} bean
+     * @param {String} id
+
+     * @private
+     */
+    static constructBean(sb, bean, vars = [], limitLines = false, id = bean.id) {
+        _.forEach(bean.arguments, (arg) => {
+            switch (arg.clsName) {
+                case 'MAP':
+                    this._constructMap(sb, arg, vars);
 
-                        sb.emptyLine();
+                    sb.emptyLine();
 
-                        break;
-                    default:
-                        if (this._isBean(arg.clsName) && arg.value.isComplex()) {
-                            this.constructBean(sb, arg.value, vars, limitLines);
+                    break;
+                default:
+                    if (this._isBean(arg.clsName) && arg.value.isComplex()) {
+                        this.constructBean(sb, arg.value, vars, limitLines);
 
-                            sb.emptyLine();
-                        }
-                }
-            });
+                        sb.emptyLine();
+                    }
+            }
+        });
 
-            const clsName = JavaTypes.shortClassName(bean.clsName);
+        const clsName = this.javaTypes.shortClassName(bean.clsName);
 
-            sb.append(`${this.varInit(clsName, id, vars)} = ${this._newBean(bean)};`);
+        sb.append(`${this.varInit(clsName, id, vars)} = ${this._newBean(bean)};`);
 
-            if (_.nonEmpty(bean.properties)) {
-                sb.emptyLine();
+        if (_.nonEmpty(bean.properties)) {
+            sb.emptyLine();
 
-                this._setProperties(sb, bean, vars, limitLines, id);
-            }
+            this._setProperties(sb, bean, vars, limitLines, id);
         }
+    }
 
-        /**
-         * @param {StringBuilder} sb
-         * @param {Bean} bean
-         * @param {Array.<String>} vars
-         * @param {Boolean} limitLines
-         * @private
-         */
-        static constructStoreFactory(sb, bean, vars, limitLines = false) {
-            const shortClsName = JavaTypes.shortClassName(bean.clsName);
-
-            if (_.includes(vars, bean.id))
-                sb.append(`${bean.id} = ${this._newBean(bean)};`);
-            else {
-                vars.push(bean.id);
-
-                sb.append(`${shortClsName} ${bean.id} = ${this._newBean(bean)};`);
-            }
-
-            sb.emptyLine();
+    /**
+     * @param {StringBuilder} sb
+     * @param {Bean} bean
+     * @param {Array.<String>} vars
+     * @param {Boolean} limitLines
+     * @private
+     */
+    static constructStoreFactory(sb, bean, vars, limitLines = false) {
+        const shortClsName = this.javaTypes.shortClassName(bean.clsName);
+
+        if (_.includes(vars, bean.id))
+            sb.append(`${bean.id} = ${this._newBean(bean)};`);
+        else {
+            vars.push(bean.id);
+
+            sb.append(`${shortClsName} ${bean.id} = ${this._newBean(bean)};`);
+        }
 
-            sb.startBlock(`${bean.id}.setDataSourceFactory(new Factory<DataSource>() {`);
-            this.commentBlock(sb, '{@inheritDoc}');
-            sb.startBlock('@Override public DataSource create() {');
+        sb.emptyLine();
 
-            sb.append(`return DataSources.INSTANCE_${bean.findProperty('dataSourceBean').id};`);
+        sb.startBlock(`${bean.id}.setDataSourceFactory(new Factory<DataSource>() {`);
+        this.commentBlock(sb, '{@inheritDoc}');
+        sb.startBlock('@Override public DataSource create() {');
 
-            sb.endBlock('};');
-            sb.endBlock('});');
+        sb.append(`return DataSources.INSTANCE_${bean.findProperty('dataSourceBean').id};`);
 
-            const storeFactory = _.cloneDeep(bean);
+        sb.endBlock('};');
+        sb.endBlock('});');
 
-            _.remove(storeFactory.properties, (p) => _.includes(['dataSourceBean'], p.name));
+        const storeFactory = _.cloneDeep(bean);
 
-            if (storeFactory.properties.length) {
-                sb.emptyLine();
+        _.remove(storeFactory.properties, (p) => _.includes(['dataSourceBean'], p.name));
 
-                this._setProperties(sb, storeFactory, vars, limitLines);
-            }
-        }
+        if (storeFactory.properties.length) {
+            sb.emptyLine();
 
-        static _isBean(clsName) {
-            return JavaTypes.nonBuiltInClass(clsName) && JavaTypes.nonEnum(clsName) && _.includes(clsName, '.');
+            this._setProperties(sb, storeFactory, vars, limitLines);
         }
+    }
 
-        static _toObject(clsName, val) {
-            const items = _.isArray(val) ? val : [val];
+    static _isBean(clsName) {
+        return this.javaTypes.nonBuiltInClass(clsName) && this.javaTypes.nonEnum(clsName) && _.includes(clsName, '.');
+    }
 
-            return _.map(items, (item) => {
-                if (_.isNil(item))
-                    return 'null';
+    static _toObject(clsName, val) {
+        const items = _.isArray(val) ? val : [val];
+
+        return _.map(items, (item) => {
+            if (_.isNil(item))
+                return 'null';
+
+            switch (clsName) {
+                case 'var':
+                    return item;
+                case 'byte':
+                    return `(byte) ${item}`;
+                case 'float':
+                    return `${item}f`;
+                case 'long':
+                    return `${item}L`;
+                case 'java.io.Serializable':
+                case 'java.lang.String':
+                    return `"${item.replace(/\\/g, '\\\\').replace(/"/g, '\\"')}"`;
+                case 'PATH':
+                    return `"${item.replace(/\\/g, '\\\\')}"`;
+                case 'java.lang.Class':
+                    return `${this.javaTypes.shortClassName(item)}.class`;
+                case 'java.util.UUID':
+                    return `UUID.fromString("${item}")`;
+                case 'PROPERTY':
+                    return `props.getProperty("${item}")`;
+                case 'PROPERTY_CHAR':
+                    return `props.getProperty("${item}").toCharArray()`;
+                case 'PROPERTY_INT':
+                    return `Integer.parseInt(props.getProperty("${item}"))`;
+                default:
+                    if (this._isBean(clsName)) {
+                        if (item.isComplex())
+                            return item.id;
+
+                        return this._newBean(item);
+                    }
 
-                switch (clsName) {
-                    case 'var':
+                    if (this.javaTypes.nonEnum(clsName))
                         return item;
-                    case 'byte':
-                        return `(byte) ${item}`;
-                    case 'float':
-                        return `${item}f`;
-                    case 'long':
-                        return `${item}L`;
-                    case 'java.io.Serializable':
-                    case 'java.lang.String':
-                        return `"${item.replace(/\\/g, '\\\\').replace(/"/g, '\\"')}"`;
-                    case 'PATH':
-                        return `"${item.replace(/\\/g, '\\\\')}"`;
-                    case 'java.lang.Class':
-                        return `${JavaTypes.shortClassName(item)}.class`;
-                    case 'java.util.UUID':
-                        return `UUID.fromString("${item}")`;
-                    case 'PROPERTY':
-                        return `props.getProperty("${item}")`;
-                    case 'PROPERTY_CHAR':
-                        return `props.getProperty("${item}").toCharArray()`;
-                    case 'PROPERTY_INT':
-                        return `Integer.parseInt(props.getProperty("${item}"))`;
-                    default:
-                        if (this._isBean(clsName)) {
-                            if (item.isComplex())
-                                return item.id;
-
-                            return this._newBean(item);
-                        }
 
-                        if (JavaTypes.nonEnum(clsName))
-                            return item;
-
-                        return `${JavaTypes.shortClassName(clsName)}.${item}`;
-                }
-            });
-        }
+                    return `${this.javaTypes.shortClassName(clsName)}.${item}`;
+            }
+        });
+    }
 
-        static _constructBeans(sb, type, items, vars, limitLines) {
-            if (this._isBean(type)) {
-                // Construct objects inline for preview or simple objects.
-                const mapper = this.METHOD_MAPPING[type];
+    static _mapperId(mapper) {
+        return (item) => this.javaTypes.toJavaName(mapper.prefix, item.findProperty(mapper.name).value);
+    }
 
-                const nextId = mapper ? mapper.id : beenNameSeed();
+    static _constructBeans(sb, type, items, vars, limitLines) {
+        if (this._isBean(type)) {
+            // Construct objects inline for preview or simple objects.
+            const mapper = this.METHOD_MAPPING[type];
 
-                // Prepare objects refs.
-                return _.map(items, (item) => {
-                    if (limitLines && mapper)
-                        return mapper.id(item) + (limitLines ? `(${mapper.args})` : '');
+            const nextId = mapper ? this._mapperId(mapper) : beanNameSeed();
 
-                    if (item.isComplex()) {
-                        const id = nextId(item);
+            // Prepare objects refs.
+            return _.map(items, (item) => {
+                if (limitLines && mapper)
+                    return nextId(item) + (limitLines ? `(${mapper.args})` : '');
 
-                        this.constructBean(sb, item, vars, limitLines, id);
+                if (item.isComplex()) {
+                    const id = nextId(item);
 
-                        sb.emptyLine();
+                    this.constructBean(sb, item, vars, limitLines, id);
 
-                        return id;
-                    }
+                    sb.emptyLine();
 
-                    return this._newBean(item);
-                });
-            }
+                    return id;
+                }
 
-            return this._toObject(type, items);
+                return this._newBean(item);
+            });
         }
 
-        /**
-         *
-         * @param sb
-         * @param parentId
-         * @param arrProp
-         * @param vars
-         * @param limitLines
-         * @private
-         */
-        static _setVarArg(sb, parentId, arrProp, vars, limitLines) {
-            const refs = this._constructBeans(sb, arrProp.typeClsName, arrProp.items, vars, limitLines);
-
-            // Set refs to property.
-            if (refs.length === 1)
-                this._setProperty(sb, parentId, arrProp.name, _.head(refs));
-            else {
-                sb.startBlock(`${parentId}.set${_.upperFirst(arrProp.name)}(`);
-
-                const lastIdx = refs.length - 1;
-
-                _.forEach(refs, (ref, idx) => {
-                    sb.append(ref + (lastIdx !== idx ? ',' : ''));
-                });
+        return this._toObject(type, items);
+    }
 
-                sb.endBlock(');');
-            }
+    /**
+     *
+     * @param sb
+     * @param parentId
+     * @param arrProp
+     * @param vars
+     * @param limitLines
+     * @private
+     */
+    static _setVarArg(sb, parentId, arrProp, vars, limitLines) {
+        const refs = this._constructBeans(sb, arrProp.typeClsName, arrProp.items, vars, limitLines);
+
+        // Set refs to property.
+        if (refs.length === 1)
+            this._setProperty(sb, parentId, arrProp.name, _.head(refs));
+        else {
+            sb.startBlock(`${parentId}.set${_.upperFirst(arrProp.name)}(`);
+
+            const lastIdx = refs.length - 1;
+
+            _.forEach(refs, (ref, idx) => {
+                sb.append(ref + (lastIdx !== idx ? ',' : ''));
+            });
+
+            sb.endBlock(');');
         }
+    }
 
-        /**
-         *
-         * @param sb
-         * @param parentId
-         * @param arrProp
-         * @param vars
-         * @param limitLines
-         * @private
-         */
-        static _setArray(sb, parentId, arrProp, vars, limitLines) {
-            const refs = this._constructBeans(sb, arrProp.typeClsName, arrProp.items, vars, limitLines);
+    /**
+     *
+     * @param sb
+     * @param parentId
+     * @param arrProp
+     * @param vars
+     * @param limitLines
+     * @private
+     */
+    static _setArray(sb, parentId, arrProp, vars, limitLines) {
+        const refs = this._constructBeans(sb, arrProp.typeClsName, arrProp.items, vars, limitLines);
 
-            const arrType = JavaTypes.shortClassName(arrProp.typeClsName);
+        const arrType = this.javaTypes.shortClassName(arrProp.typeClsName);
 
-            // Set refs to property.
-            sb.startBlock(`${parentId}.set${_.upperFirst(arrProp.name)}(new ${arrType}[] {`);
+        // Set refs to property.
+        sb.startBlock(`${parentId}.set${_.upperFirst(arrProp.name)}(new ${arrType}[] {`);
 
-            const lastIdx = refs.length - 1;
+        const lastIdx = refs.length - 1;
 
-            _.forEach(refs, (ref, idx) => sb.append(ref + (lastIdx !== idx ? ',' : '')));
+        _.forEach(refs, (ref, idx) => sb.append(ref + (lastIdx !== idx ? ',' : '')));
 
-            sb.endBlock('});');
-        }
+        sb.endBlock('});');
+    }
 
-        static _constructMap(sb, map, vars = []) {
-            const keyClsName = JavaTypes.shortClassName(map.keyClsName);
-            const valClsName = JavaTypes.shortClassName(map.valClsName);
+    static _constructMap(sb, map, vars = []) {
+        const keyClsName = this.javaTypes.shortClassName(map.keyClsName);
+        const valClsName = this.javaTypes.shortClassName(map.valClsName);
 
-            const mapClsName = map.ordered ? 'LinkedHashMap' : 'HashMap';
+        const mapClsName = map.ordered ? 'LinkedHashMap' : 'HashMap';
 
-            const type = `${mapClsName}<${keyClsName}, ${valClsName}>`;
+        const type = `${mapClsName}<${keyClsName}, ${valClsName}>`;
 
-            sb.append(`${this.varInit(type, map.id, vars)} = new ${mapClsName}<>();`);
+        sb.append(`${this.varInit(type, map.id, vars)} = new ${mapClsName}<>();`);
 
-            sb.emptyLine();
+        sb.emptyLine();
 
-            _.forEach(map.entries, (entry) => {
-                const key = this._toObject(map.keyClsName, entry[map.keyField]);
-                const val = entry[map.valField];
+        _.forEach(map.entries, (entry) => {
+            const key = this._toObject(map.keyClsName, entry[map.keyField]);
+            const val = entry[map.valField];
 
-                if (_.isArray(val) && map.valClsName === 'java.lang.String') {
-                    if (val.length > 1) {
-                        sb.startBlock(`${map.id}.put(${key},`);
+            if (_.isArray(val) && map.valClsName === 'java.lang.String') {
+                if (val.length > 1) {
+                    sb.startBlock(`${map.id}.put(${key},`);
 
-                        _.forEach(val, (line, idx) => {
-                            sb.append(`"${line}"${idx !== val.length - 1 ? ' +' : ''}`);
-                        });
+                    _.forEach(val, (line, idx) => {
+                        sb.append(`"${line}"${idx !== val.length - 1 ? ' +' : ''}`);
+                    });
 
-                        sb.endBlock(');');
-                    }
-                    else
-                        sb.append(`${map.id}.put(${key}, ${this._toObject(map.valClsName, _.head(val))});`);
+                    sb.endBlock(');');
                 }
                 else
-                    sb.append(`${map.id}.put(${key}, ${this._toObject(map.valClsName, val)});`);
-            });
-        }
+                    sb.append(`${map.id}.put(${key}, ${this._toObject(map.valClsName, _.head(val))});`);
+            }
+            else
+                sb.append(`${map.id}.put(${key}, ${this._toObject(map.valClsName, val)});`);
+        });
+    }
 
-        static varInit(type, id, vars) {
-            if (_.includes(vars, id))
-                return id;
+    static varInit(type, id, vars) {
+        if (_.includes(vars, id))
+            return id;
 
-            vars.push(id);
+        vars.push(id);
 
-            return `${type} ${id}`;
-        }
+        return `${type} ${id}`;
+    }
 
-        /**
-         *
-         * @param {StringBuilder} sb
-         * @param {Bean} bean
-         * @param {String} id
-         * @param {Array.<String>} vars
-         * @param {Boolean} limitLines
-         * @returns {StringBuilder}
-         */
-        static _setProperties(sb = new StringBuilder(), bean, vars = [], limitLines = false, id = bean.id) {
-            _.forEach(bean.properties, (prop, idx) => {
-                switch (prop.clsName) {
-                    case 'DATA_SOURCE':
-                        this._setProperty(sb, id, 'dataSource', `DataSources.INSTANCE_${prop.id}`);
-
-                        break;
-                    case 'EVENT_TYPES':
-                        if (prop.eventTypes.length === 1)
-                            this._setProperty(sb, id, prop.name, _.head(prop.eventTypes));
-                        else {
-                            sb.append(`int[] ${prop.id} = new int[${_.head(prop.eventTypes)}.length`);
+    /**
+     *
+     * @param {StringBuilder} sb
+     * @param {Bean} bean
+     * @param {String} id
+     * @param {Array.<String>} vars
+     * @param {Boolean} limitLines
+     * @returns {StringBuilder}
+     */
+    static _setProperties(sb = new StringBuilder(), bean, vars = [], limitLines = false, id = bean.id) {
+        _.forEach(bean.properties, (prop, idx) => {
+            switch (prop.clsName) {
+                case 'DATA_SOURCE':
+                    this._setProperty(sb, id, 'dataSource', `DataSources.INSTANCE_${prop.id}`);
+
+                    break;
+                case 'EVENT_TYPES':
+                    if (prop.eventTypes.length === 1)
+                        this._setProperty(sb, id, prop.name, _.head(prop.eventTypes));
+                    else {
+                        sb.append(`int[] ${prop.id} = new int[${_.head(prop.eventTypes)}.length`);
+
+                        _.forEach(_.tail(prop.eventTypes), (evtGrp) => {
+                            sb.append(`    + ${evtGrp}.length`);
+                        });
 
-                            _.forEach(_.tail(prop.eventTypes), (evtGrp) => {
-                                sb.append(`    + ${evtGrp}.length`);
-                            });
+                        sb.append('];');
 
-                            sb.append('];');
+                        sb.emptyLine();
 
+                        sb.append('int k = 0;');
+
+                        _.forEach(prop.eventTypes, (evtGrp, evtIdx) => {
                             sb.emptyLine();
 
-                            sb.append('int k = 0;');
+                            sb.append(`System.arraycopy(${evtGrp}, 0, ${prop.id}, k, ${evtGrp}.length);`);
 
-                            _.forEach(prop.eventTypes, (evtGrp, evtIdx) => {
-                                sb.emptyLine();
+                            if (evtIdx < prop.eventTypes.length - 1)
+                                sb.append(`k += ${evtGrp}.length;`);
+                        });
 
-                                sb.append(`System.arraycopy(${evtGrp}, 0, ${prop.id}, k, ${evtGrp}.length);`);
+                        sb.emptyLine();
 
-                                if (evtIdx < prop.eventTypes.length - 1)
-                                    sb.append(`k += ${evtGrp}.length;`);
-                            });
+                        sb.append(`cfg.setIncludeEventTypes(${prop.id});`);
+                    }
 
-                            sb.emptyLine();
+                    break;
+                case 'ARRAY':
+                    if (prop.varArg)
+                        this._setVarArg(sb, id, prop, vars, limitLines);
+                    else
+                        this._setArray(sb, id, prop, vars, limitLines);
 
-                            sb.append(`cfg.setIncludeEventTypes(${prop.id});`);
-                        }
+                    break;
+                case 'COLLECTION':
+                    const nonBean = !this._isBean(prop.typeClsName);
 
-                        break;
-                    case 'ARRAY':
-                        if (prop.varArg)
-                            this._setVarArg(sb, id, prop, vars, limitLines);
-                        else
-                            this._setArray(sb, id, prop, vars, limitLines);
+                    if (nonBean && prop.implClsName === 'java.util.ArrayList') {
+                        const items = _.map(prop.items, (item) => this._toObject(prop.typeClsName, item));
 
-                        break;
-                    case 'COLLECTION':
-                        const nonBean = !this._isBean(prop.typeClsName);
+                        if (items.length > 1) {
+                            sb.startBlock(`${id}.set${_.upperFirst(prop.name)}(Arrays.asList(`);
 
-                        if (nonBean && prop.implClsName === 'java.util.ArrayList') {
-                            const items = _.map(prop.items, (item) => this._toObject(prop.typeClsName, item));
+                            _.forEach(items, (item, i) => sb.append(item + (i !== items.length - 1 ? ',' : '')));
 
-                            if (items.length > 1) {
-                                sb.startBlock(`${id}.set${_.upperFirst(prop.name)}(Arrays.asList(`);
+                            sb.endBlock('));');
+                        }
+                        else
+                            this._setProperty(sb, id, prop.name, `Arrays.asList(${items})`);
+                    }
+                    else {
+                        const colTypeClsName = this.javaTypes.shortClassName(prop.typeClsName);
+                        const implClsName = this.javaTypes.shortClassName(prop.implClsName);
 
-                                _.forEach(items, (item, i) => sb.append(item + (i !== items.length - 1 ? ',' : '')));
+                        sb.append(`${this.varInit(`${implClsName}<${colTypeClsName}>`, prop.id, vars)} = new ${implClsName}<>();`);
 
-                                sb.endBlock('));');
-                            }
-                            else
-                                this._setProperty(sb, id, prop.name, `Arrays.asList(${items})`);
+                        sb.emptyLine();
+
+                        if (nonBean) {
+                            _.forEach(this._toObject(colTypeClsName, prop.items), (item) => {
+                                sb.append(`${prop.id}.add("${item}");`);
+
+                                sb.emptyLine();
+                            });
                         }
                         else {
-                            const colTypeClsName = JavaTypes.shortClassName(prop.typeClsName);
-                            const implClsName = JavaTypes.shortClassName(prop.implClsName);
+                            _.forEach(prop.items, (item) => {
+                                this.constructBean(sb, item, vars, limitLines);
 
-                            sb.append(`${this.varInit(`${implClsName}<${colTypeClsName}>`, prop.id, vars)} = new ${implClsName}<>();`);
+                                sb.append(`${prop.id}.add(${item.id});`);
 
-                            sb.emptyLine();
+                                sb.emptyLine();
+                            });
+                        }
 
-                            if (nonBean) {
-                                _.forEach(this._toObject(colTypeClsName, prop.items), (item) => {
-                                    sb.append(`${prop.id}.add("${item}");`);
+                        this._setProperty(sb, id, prop.name, prop.id);
+                    }
 
-                                    sb.emptyLine();
-                                });
-                            }
-                            else {
-                                _.forEach(prop.items, (item) => {
-                                    this.constructBean(sb, item, vars, limitLines);
+                    break;
+                case 'MAP':
+                    this._constructMap(sb, prop, vars);
 
-                                    sb.append(`${prop.id}.add(${item.id});`);
+                    if (_.nonEmpty(prop.entries))
+                        sb.emptyLine();
 
-                                    sb.emptyLine();
-                                });
-                            }
+                    this._setProperty(sb, id, prop.name, prop.id);
 
-                            this._setProperty(sb, id, prop.name, prop.id);
-                        }
+                    break;
+                case 'java.util.Properties':
+                    sb.append(`${this.varInit('Properties', prop.id, vars)} = new Properties();`);
 
-                        break;
-                    case 'MAP':
-                        this._constructMap(sb, prop, vars);
+                    if (_.nonEmpty(prop.entries))
+                        sb.emptyLine();
 
-                        if (_.nonEmpty(prop.entries))
-                            sb.emptyLine();
+                    _.forEach(prop.entries, (entry) => {
+                        const key = this._toObject('java.lang.String', entry.name);
+                        const val = this._toObject('java.lang.String', entry.value);
 
-                        this._setProperty(sb, id, prop.name, prop.id);
+                        sb.append(`${prop.id}.setProperty(${key}, ${val});`);
+                    });
 
-                        break;
-                    case 'java.util.Properties':
-                        sb.append(`${this.varInit('Properties', prop.id, vars)} = new Properties();`);
+                    sb.emptyLine();
 
-                        if (_.nonEmpty(prop.entries))
-                            sb.emptyLine();
+                    this._setProperty(sb, id, prop.name, prop.id);
 
-                        _.forEach(prop.entries, (entry) => {
-                            const key = this._toObject('java.lang.String', entry.name);
-                            const val = this._toObject('java.lang.String', entry.value);
+                    break;
+                case 'BEAN':
+                    const embedded = prop.value;
 
-                            sb.append(`${prop.id}.setProperty(${key}, ${val});`);
-                        });
+                    if (_.includes(STORE_FACTORY, embedded.clsName)) {
+                        this.constructStoreFactory(sb, embedded, vars, limitLines);
 
                         sb.emptyLine();
 
-                        this._setProperty(sb, id, prop.name, prop.id);
+                        this._setProperty(sb, id, prop.name, embedded.id);
+                    }
+                    else if (embedded.isComplex()) {
+                        this.constructBean(sb, embedded, vars, limitLines);
 
-                        break;
-                    case 'BEAN':
-                        const embedded = prop.value;
+                        sb.emptyLine();
 
-                        if (_.includes(STORE_FACTORY, embedded.clsName)) {
-                            this.constructStoreFactory(sb, embedded, vars, limitLines);
+                        this._setProperty(sb, id, prop.name, embedded.id);
+                    }
+                    else
+                        this._setProperty(sb, id, prop.name, this._newBean(embedded));
 
-                            sb.emptyLine();
+                    break;
+                default:
+                    this._setProperty(sb, id, prop.name, this._toObject(prop.clsName, prop.value));
+            }
 
-                            this._setProperty(sb, id, prop.name, embedded.id);
-                        }
-                        else if (embedded.isComplex()) {
-                            this.constructBean(sb, embedded, vars, limitLines);
+            this._emptyLineIfNeeded(sb, bean.properties, idx);
+        });
 
-                            sb.emptyLine();
+        return sb;
+    }
 
-                            this._setProperty(sb, id, prop.name, embedded.id);
-                        }
-                        else
-                            this._setProperty(sb, id, prop.name, this._newBean(embedded));
+    static _collectMapImports(prop) {
+        const imports = [];
 
-                        break;
-                    default:
-                        this._setProperty(sb, id, prop.name, this._toObject(prop.clsName, prop.value));
-                }
+        imports.push(prop.ordered ? 'java.util.LinkedHashMap' : 'java.util.HashMap');
+        imports.push(prop.keyClsName);
+        imports.push(prop.valClsName);
 
-                this._emptyLineIfNeeded(sb, bean.properties, idx);
-            });
+        return imports;
+    }
 
-            return sb;
-        }
+    static collectBeanImports(bean) {
+        const imports = [bean.clsName];
 
-        static collectBeanImports(bean) {
-            const imports = [bean.clsName];
+        _.forEach(bean.arguments, (arg) => {
+            switch (arg.clsName) {
+                case 'BEAN':
+                    imports.push(...this.collectPropertiesImports(arg.value.properties));
 
-            _.forEach(bean.arguments, (arg) => {
-                switch (arg.clsName) {
-                    case 'BEAN':
-                        imports.push(...this.collectPropertiesImports(arg.value.properties));
+                    break;
+                case 'java.lang.Class':
+                    imports.push(this.javaTypes.fullClassName(arg.value));
 
-                        break;
-                    case 'java.lang.Class':
-                        imports.push(JavaTypes.fullClassName(arg.value));
+                    break;
 
-                        break;
-                    default:
-                        imports.push(arg.clsName);
-                }
-            });
+                case 'MAP':
+                    imports.push(...this._collectMapImports(arg));
 
-            imports.push(...this.collectPropertiesImports(bean.properties));
+                    break;
+                default:
+                    imports.push(arg.clsName);
+            }
+        });
 
-            if (_.includes(STORE_FACTORY, bean.clsName))
-                imports.push('javax.sql.DataSource', 'javax.cache.configuration.Factory');
+        imports.push(...this.collectPropertiesImports(bean.properties));
 
-            return imports;
-        }
+        if (_.includes(STORE_FACTORY, bean.clsName))
+            imports.push('javax.sql.DataSource', 'javax.cache.configuration.Factory');
 
-        /**
-         * @param {Array.<Object>} props
-         * @returns {Array.<String>}
-         */
-        static collectPropertiesImports(props) {
-            const imports = [];
+        return imports;
+    }
 
-            _.forEach(props, (prop) => {
-                switch (prop.clsName) {
-                    case 'DATA_SOURCE':
-                        imports.push(prop.value.clsName);
+    /**
+     * @param {Array.<Object>} props
+     * @returns {Array.<String>}
+     */
+    static collectPropertiesImports(props) {
+        const imports = [];
 
-                        break;
-                    case 'PROPERTY':
-                    case 'PROPERTY_CHAR':
-                    case 'PROPERTY_INT':
-                        imports.push('java.io.InputStream', 'java.util.Properties');
+        _.forEach(props, (prop) => {
+            switch (prop.clsName) {
+                case 'DATA_SOURCE':
+                    imports.push(prop.value.clsName);
 
-                        break;
-                    case 'BEAN':
-                        imports.push(...this.collectBeanImports(prop.value));
+                    break;
+                case 'PROPERTY':
+                case 'PROPERTY_CHAR':
+                case 'PROPERTY_INT':
+                    imports.push('java.io.InputStream', 'java.util.Properties');
 
-                        break;
-                    case 'ARRAY':
-                        imports.push(prop.typeClsName);
+                    break;
+                case 'BEAN':
+                    imports.push(...this.collectBeanImports(prop.value));
 
-                        if (this._isBean(prop.typeClsName))
-                            _.forEach(prop.items, (item) => imports.push(...this.collectBeanImports(item)));
+                    break;
+                case 'ARRAY':
+                    imports.push(prop.typeClsName);
 
-                        break;
-                    case 'COLLECTION':
-                        imports.push(prop.typeClsName);
+                    if (this._isBean(prop.typeClsName))
+                        _.forEach(prop.items, (item) => imports.push(...this.collectBeanImports(item)));
 
-                        if (this._isBean(prop.typeClsName)) {
-                            _.forEach(prop.items, (item) => imports.push(...this.collectBeanImports(item)));
+                    break;
+                case 'COLLECTION':
+                    imports.push(prop.typeClsName);
 
-                            imports.push(prop.implClsName);
-                        }
-                        else if (prop.implClsName === 'java.util.ArrayList')
-                            imports.push('java.util.Arrays');
-                        else
-                            imports.push(prop.implClsName);
-
-                        break;
-                    case 'MAP':
-                        imports.push(prop.ordered ? 'java.util.LinkedHashMap' : 'java.util.HashMap');
-                        imports.push(prop.keyClsName);
-                        imports.push(prop.valClsName);
-
-                        break;
-                    default:
-                        if (!JavaTypes.nonEnum(prop.clsName))
-                            imports.push(prop.clsName);
-                }
-            });
+                    if (this._isBean(prop.typeClsName)) {
+                        _.forEach(prop.items, (item) => imports.push(...this.collectBeanImports(item)));
 
-            return imports;
-        }
+                        imports.push(prop.implClsName);
+                    }
+                    else if (prop.implClsName === 'java.util.ArrayList')
+                        imports.push('java.util.Arrays');
+                    else
+                        imports.push(prop.implClsName);
 
-        static _prepareImports(imports) {
-            return _.sortedUniq(_.sortBy(_.filter(imports, (cls) => !cls.startsWith('java.lang.') && _.includes(cls, '.'))));
-        }
+                    break;
+                case 'MAP':
+                    imports.push(...this._collectMapImports(prop));
 
-        /**
-         * @param {Bean} bean
-         * @returns {Array.<String>}
-         */
-        static collectStaticImports(bean) {
-            const imports = [];
+                    break;
+                default:
+                    if (!this.javaTypes.nonEnum(prop.clsName))
+                        imports.push(prop.clsName);
+            }
+        });
 
-            _.forEach(bean.properties, (prop) => {
-                switch (prop.clsName) {
-                    case 'EVENT_TYPES':
-                        _.forEach(prop.eventTypes, (value) => {
-                            const evtGrp = _.find(eventGroups, {value});
+        return imports;
+    }
 
-                            imports.push(`${evtGrp.class}.${evtGrp.value}`);
-                        });
+    static _prepareImports(imports) {
+        return _.sortedUniq(_.sortBy(_.filter(imports, (cls) => !cls.startsWith('java.lang.') && _.includes(cls, '.'))));
+    }
 
-                        break;
-                    default:
-                        // No-op.
-                }
-            });
+    /**
+     * @param {Bean} bean
+     * @returns {Array.<String>}
+     */
+    static collectStaticImports(bean) {
+        const imports = [];
 
-            return imports;
-        }
+        _.forEach(bean.properties, (prop) => {
+            switch (prop.clsName) {
+                case 'EVENT_TYPES':
+                    _.forEach(prop.eventTypes, (value) => {
+                        const evtGrp = _.find(this.eventGroups, {value});
 
-        /**
-         * @param {Bean} bean
-         * @returns {Object}
-         */
-        static collectBeansWithMapping(bean) {
-            const beans = {};
+                        imports.push(`${evtGrp.class}.${evtGrp.value}`);
+                    });
 
-            _.forEach(bean.properties, (prop) => {
-                switch (prop.clsName) {
-                    case 'BEAN':
-                        _.merge(beans, this.collectBeansWithMapping(prop.value));
+                    break;
+                default:
+                    // No-op.
+            }
+        });
 
-                        break;
-                    case 'ARRAY':
-                        if (this._isBean(prop.typeClsName)) {
-                            const mapping = this.METHOD_MAPPING[prop.typeClsName];
+        return imports;
+    }
 
-                            _.reduce(prop.items, (acc, item) => {
-                                if (mapping) {
-                                    acc[mapping.id(item)] = item;
+    /**
+     * @param {Bean} bean
+     * @returns {Object}
+     */
+    static collectBeansWithMapping(bean) {
+        const beans = {};
 
-                                    _.merge(acc, this.collectBeansWithMapping(item));
-                                }
-                                return acc;
-                            }, beans);
-                        }
+        _.forEach(bean.properties, (prop) => {
+            switch (prop.clsName) {
+                case 'BEAN':
+                    _.merge(beans, this.collectBeansWithMapping(prop.value));
 
-                        break;
-                    default:
-                        // No-op.
-                }
-            });
+                    break;
+                case 'ARRAY':
+                    if (this._isBean(prop.typeClsName)) {
+                        const mapper = this.METHOD_MAPPING[prop.typeClsName];
 
-            return beans;
-        }
+                        const mapperId = mapper ? this._mapperId(mapper) : null;
 
-        /**
-         * Build Java startup class with configuration.
-         *
-         * @param {Bean} cfg
-         * @param pkg Package name.
-         * @param {String} clsName Class name for generate factory class otherwise generate code snippet.
-         * @param {Array.<Object>} clientNearCaches Is client node.
-         * @returns {StringBuilder}
-         */
-        static igniteConfiguration(cfg, pkg, clsName, clientNearCaches) {
-            const sb = new StringBuilder();
-
-            sb.append(`package ${pkg};`);
-            sb.emptyLine();
+                        _.reduce(prop.items, (acc, item) => {
+                            if (mapperId)
+                                acc[mapperId(item)] = item;
 
-            const imports = this.collectBeanImports(cfg);
+                            _.merge(acc, this.collectBeansWithMapping(item));
 
-            if (_.nonEmpty(clientNearCaches))
-                imports.push('org.apache.ignite.configuration.NearCacheConfiguration');
+                            return acc;
+                        }, beans);
+                    }
 
-            if (_.includes(imports, 'oracle.jdbc.pool.OracleDataSource'))
-                imports.push('java.sql.SQLException');
+                    break;
+                default:
+                    // No-op.
+            }
+        });
 
-            const hasProps = this.hasProperties(cfg);
+        return beans;
+    }
 
-            if (hasProps)
-                imports.push('java.util.Properties', 'java.io.InputStream');
+    /**
+     * Build Java startup class with configuration.
+     *
+     * @param {Bean} cfg
+     * @param pkg Package name.
+     * @param {String} clsName Class name for generate factory class otherwise generate code snippet.
+     * @param {Array.<Object>} clientNearCaches Is client node.
+     * @returns {StringBuilder}
+     */
+    static igniteConfiguration(cfg, pkg, clsName, clientNearCaches) {
+        const sb = new StringBuilder();
 
-            _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`));
+        sb.append(`package ${pkg};`);
+        sb.emptyLine();
 
-            sb.emptyLine();
+        const imports = this.collectBeanImports(cfg);
 
-            const staticImports = this._prepareImports(this.collectStaticImports(cfg));
+        const nearCacheBeans = [];
 
-            if (staticImports.length) {
-                _.forEach(this._prepareImports(staticImports), (cls) => sb.append(`import static ${cls};`));
+        if (_.nonEmpty(clientNearCaches)) {
+            imports.push('org.apache.ignite.configuration.NearCacheConfiguration');
 
-                sb.emptyLine();
-            }
+            _.forEach(clientNearCaches, (cache) => {
+                const nearCacheBean = this.generator.cacheNearClient(cache);
 
-            this.mainComment(sb);
-            sb.startBlock(`public class ${clsName} {`);
+                nearCacheBean.cacheName = cache.name;
 
-            // 2. Add external property file
-            if (hasProps) {
-                this.commentBlock(sb, 'Secret properties loading.');
-                sb.append('private static final Properties props = new Properties();');
-                sb.emptyLine();
-                sb.startBlock('static {');
-                sb.startBlock('try (InputStream in = IgniteConfiguration.class.getClassLoader().getResourceAsStream("secret.properties")) {');
-                sb.append('props.load(in);');
-                sb.endBlock('}');
-                sb.startBlock('catch (Exception ignored) {');
-                sb.append('// No-op.');
-                sb.endBlock('}');
-                sb.endBlock('}');
-                sb.emptyLine();
-            }
+                imports.push(...this.collectBeanImports(nearCacheBean));
 
-            // 3. Add data sources.
-            const dataSources = this.collectDataSources(cfg);
+                nearCacheBeans.push(nearCacheBean);
+            });
+        }
 
-            if (dataSources.length) {
-                this.commentBlock(sb, 'Helper class for datasource creation.');
-                sb.startBlock('public static class DataSources {');
+        if (_.includes(imports, 'oracle.jdbc.pool.OracleDataSource'))
+            imports.push('java.sql.SQLException');
 
-                _.forEach(dataSources, (ds, idx) => {
-                    const dsClsName = JavaTypes.shortClassName(ds.clsName);
+        const hasProps = this.hasProperties(cfg);
 
-                    if (idx !== 0)
-                        sb.emptyLine();
+        if (hasProps)
+            imports.push('java.util.Properties', 'java.io.InputStream');
 
-                    sb.append(`public static final ${dsClsName} INSTANCE_${ds.id} = create${ds.id}();`);
-                    sb.emptyLine();
+        _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`));
 
-                    sb.startBlock(`private static ${dsClsName} create${ds.id}() {`);
+        sb.emptyLine();
 
-                    if (dsClsName === 'OracleDataSource')
-                        sb.startBlock('try {');
+        const staticImports = this._prepareImports(this.collectStaticImports(cfg));
 
-                    this.constructBean(sb, ds);
+        if (staticImports.length) {
+            _.forEach(this._prepareImports(staticImports), (cls) => sb.append(`import static ${cls};`));
 
-                    sb.emptyLine();
-                    sb.append(`return ${ds.id};`);
+            sb.emptyLine();
+        }
 
-                    if (dsClsName === 'OracleDataSource') {
-                        sb.endBlock('}');
-                        sb.startBlock('catch (SQLException ex) {');
-                        sb.append('throw new Error(ex);');
-                        sb.endBlock('}');
-                    }
+        this.mainComment(sb);
+        sb.startBlock(`public class ${clsName} {`);
 
-                    sb.endBlock('}');
-                });
+        // 2. Add external property file
+        if (hasProps) {
+            this.commentBlock(sb, 'Secret properties loading.');
+            sb.append('private static final Properties props = new Properties();');
+            sb.emptyLine();
+            sb.startBlock('static {');
+            sb.startBlock('try (InputStream in = IgniteConfiguration.class.getClassLoader().getResourceAsStream("secret.properties")) {');
+            sb.append('props.load(in);');
+            sb.endBlock('}');
+            sb.startBlock('catch (Exception ignored) {');
+            sb.append('// No-op.');
+            sb.endBlock('}');
+            sb.endBlock('}');
+            sb.emptyLine();
+        }
 
-                sb.endBlock('}');
+        // 3. Add data sources.
+        const dataSources = this.collectDataSources(cfg);
+
+        if (dataSources.length) {
+            this.commentBlock(sb, 'Helper class for datasource creation.');
+            sb.startBlock('public static class DataSources {');
+
+            _.forEach(dataSources, (ds, idx) => {
+                const dsClsName = this.javaTypes.shortClassName(ds.clsName);
 
+                if (idx !== 0)
+                    sb.emptyLine();
+
+                sb.append(`public static final ${dsClsName} INSTANCE_${ds.id} = create${ds.id}();`);
                 sb.emptyLine();
-            }
 
-            _.forEach(clientNearCaches, (cache) => {
-                this.commentBlock(sb, `Configuration of near cache for cache: ${cache.name}.`,
-                    '',
-                    '@return Near cache configuration.',
-                    '@throws Exception If failed to construct near cache configuration instance.'
-                );
+                sb.startBlock(`private static ${dsClsName} create${ds.id}() {`);
 
-                const nearCacheBean = generator.cacheNearClient(cache);
+                if (dsClsName === 'OracleDataSource')
+                    sb.startBlock('try {');
 
-                sb.startBlock(`public static NearCacheConfiguration ${nearCacheBean.id}() throws Exception {`);
+                this.constructBean(sb, ds);
 
-                this.constructBean(sb, nearCacheBean);
                 sb.emptyLine();
+                sb.append(`return ${ds.id};`);
 
-                sb.append(`return ${nearCacheBean.id};`);
-                sb.endBlock('}');
+                if (dsClsName === 'OracleDataSource') {
+                    sb.endBlock('}');
+                    sb.startBlock('catch (SQLException ex) {');
+                    sb.append('throw new Error(ex);');
+                    sb.endBlock('}');
+                }
 
-                sb.emptyLine();
+                sb.endBlock('}');
             });
 
-            this.commentBlock(sb, 'Configure grid.',
+            sb.endBlock('}');
+
+            sb.emptyLine();
+        }
+
+        _.forEach(nearCacheBeans, (nearCacheBean) => {
+            this.commentBlock(sb, `Configuration of near cache for cache: ${nearCacheBean.cacheName}.`,
                 '',
-                '@return Ignite configuration.',
-                '@throws Exception If failed to construct Ignite configuration instance.'
+                '@return Near cache configuration.',
+                '@throws Exception If failed to construct near cache configuration instance.'
             );
-            sb.startBlock('public static IgniteConfiguration createConfiguration() throws Exception {');
 
-            this.constructBean(sb, cfg, [], true);
+            sb.startBlock(`public static NearCacheConfiguration ${nearCacheBean.id}() throws Exception {`);
 
+            this.constructBean(sb, nearCacheBean);
             sb.emptyLine();
 
-            sb.append(`return ${cfg.id};`);
-
+            sb.append(`return ${nearCacheBean.id};`);
             sb.endBlock('}');
 
-            const beans = this.collectBeansWithMapping(cfg);
+            sb.emptyLine();
+        });
 
-            _.forEach(beans, (bean, id) => {
-                sb.emptyLine();
+        this.commentBlock(sb, 'Configure grid.',
+            '',
+            '@return Ignite configuration.',
+            '@throws Exception If failed to construct Ignite configuration instance.'
+        );
+        sb.startBlock('public static IgniteConfiguration createConfiguration() throws Exception {');
 
-                this.METHOD_MAPPING[bean.clsName].generator(sb, id, bean);
-            });
+        this.constructBean(sb, cfg, [], true);
 
-            sb.endBlock('}');
+        sb.emptyLine();
 
-            return sb;
-        }
+        sb.append(`return ${cfg.id};`);
 
-        static cluster(cluster, pkg, clsName, client) {
-            const cfg = this.generator.igniteConfiguration(cluster, client);
+        sb.endBlock('}');
 
-            const clientNearCaches = client ? _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled')) : [];
+        const beans = this.collectBeansWithMapping(cfg);
 
-            return this.igniteConfiguration(cfg, pkg, clsName, clientNearCaches);
-        }
-
-        /**
-         * Generate source code for type by its domain model.
-         *
-         * @param fullClsName Full class name.
-         * @param fields Fields.
-         * @param addConstructor If 'true' then empty and full constructors should be generated.
-         * @returns {StringBuilder}
-         */
-        static pojo(fullClsName, fields, addConstructor) {
-            const dotIdx = fullClsName.lastIndexOf('.');
+        _.forEach(beans, (bean, id) => {
+            sb.emptyLine();
 
-            const pkg = fullClsName.substring(0, dotIdx);
-            const clsName = fullClsName.substring(dotIdx + 1);
+            this.METHOD_MAPPING[bean.clsName].generator(sb, id, bean);
+        });
 
-            const sb = new StringBuilder();
+        sb.endBlock('}');
 
-            sb.append(`package ${pkg};`);
-            sb.emptyLine();
-
-            const imports = ['java.io.Serializable'];
+        return sb;
+    }
 
-            _.forEach(fields, (field) => imports.push(JavaTypes.fullClassName(field.javaFieldType)));
+    static cluster(cluster, pkg, clsName, client) {
+        const cfg = this.generator.igniteConfiguration(cluster, client);
 
-            _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`));
+        const clientNearCaches = client ? _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled')) : [];
 
-            sb.emptyLine();
+        return this.igniteConfiguration(cfg, pkg, clsName, clientNearCaches);
+    }
 
-            this.mainComment(sb,
-                `${clsName} definition.`,
-                ''
-            );
-            sb.startBlock(`public class ${clsName} implements Serializable {`);
-            sb.append('/** */');
-            sb.append('private static final long serialVersionUID = 0L;');
-            sb.emptyLine();
+    /**
+     * Generate source code for type by its domain model.
+     *
+     * @param fullClsName Full class name.
+     * @param fields Fields.
+     * @param addConstructor If 'true' then empty and full constructors should be generated.
+     * @returns {StringBuilder}
+     */
+    static pojo(fullClsName, fields, addConstructor) {
+        const dotIdx = fullClsName.lastIndexOf('.');
 
-            // Generate fields declaration.
-            _.forEach(fields, (field) => {
-                const fldName = field.javaFieldName;
-                const fldType = JavaTypes.shortClassName(field.javaFieldType);
+        const pkg = fullClsName.substring(0, dotIdx);
+        const clsName = fullClsName.substring(dotIdx + 1);
 
-                sb.append(`/** Value for ${fldName}. */`);
-                sb.append(`private ${fldType} ${fldName};`);
+        const sb = new StringBuilder();
 
-                sb.emptyLine();
-            });
+        sb.append(`package ${pkg};`);
+        sb.emptyLine();
 
-            // Generate constructors.
-            if (addConstructor) {
-                this.commentBlock(sb, 'Empty constructor.');
-                sb.startBlock(`public ${clsName}() {`);
-                this.comment(sb, 'No-op.');
-                sb.endBlock('}');
+        const imports = ['java.io.Serializable'];
 
-                sb.emptyLine();
+        _.forEach(fields, (field) => imports.push(this.javaTypes.fullClassName(field.javaFieldType)));
 
-                this.commentBlock(sb, 'Full constructor.');
+        _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`));
 
-                const arg = (field) => {
-                    const fldType = JavaTypes.shortClassName(field.javaFieldType);
+        sb.emptyLine();
 
-                    return `${fldType} ${field.javaFieldName}`;
-                };
+        this.mainComment(sb,
+            `${clsName} definition.`,
+            ''
+        );
+        sb.startBlock(`public class ${clsName} implements Serializable {`);
+        sb.append('/** */');
+        sb.append('private static final long serialVersionUID = 0L;');
+        sb.emptyLine();
 
-                sb.startBlock(`public ${clsName}(${arg(_.head(fields))}${fields.length === 1 ? ') {' : ','}`);
+        // Generate fields declaration.
+        _.forEach(fields, (field) => {
+            const fldName = field.javaFieldName;
+            const fldType = this.javaTypes.shortClassName(field.javaFieldType);
 
-                _.forEach(_.tail(fields), (field, idx) => {
-                    sb.append(`${arg(field)}${idx !== fields.length - 2 ? ',' : ') {'}`);
-                });
+            sb.append(`/** Value for ${fldName}. */`);
+            sb.append(`private ${fldType} ${fldName};`);
 
-                _.forEach(fields, (field) => sb.append(`this.${field.javaFieldName} = ${field.javaFieldName};`));
+            sb.emptyLine();
+        });
 
-                sb.endBlock('}');
+        // Generate constructors.
+        if (addConstructor) {
+            this.commentBlock(sb, 'Empty constructor.');
+            sb.startBlock(`public ${clsName}() {`);
+            this.comment(sb, 'No-op.');
+            sb.endBlock('}');
 
-                sb.emptyLine();
-            }
+            sb.emptyLine();
 
-            // Generate getters and setters methods.
-            _.forEach(fields, (field) => {
-                const fldType = JavaTypes.shortClassName(field.javaFieldType);
-                const fldName = field.javaFieldName;
+            this.commentBlock(sb, 'Full constructor.');
 
-                this.commentBlock(sb,
-                    `Gets ${fldName}`,
-                    '',
-                    `@return Value for ${fldName}.`
-                );
-                sb.startBlock(`public ${fldType} ${JavaTypes.toJavaName('get', fldName)}() {`);
-                sb.append('return ' + fldName + ';');
-                sb.endBlock('}');
+            const arg = (field) => {
+                const fldType = this.javaTypes.shortClassName(field.javaFieldType);
 
-                sb.emptyLine();
+                return `${fldType} ${field.javaFieldName}`;
+            };
 
-                this.commentBlock(sb,
-                    `Sets ${fldName}`,
-                    '',
-                    `@param ${fldName} New value for ${fldName}.`
-                );
-                sb.startBlock(`public void ${JavaTypes.toJavaName('set', fldName)}(${fldType} ${fldName}) {`);
-                sb.append(`this.${fldName} = ${fldName};`);
-                sb.endBlock('}');
+            sb.startBlock(`public ${clsName}(${arg(_.head(fields))}${fields.length === 1 ? ') {' : ','}`);
 
-                sb.emptyLine();
+            _.forEach(_.tail(fields), (field, idx) => {
+                sb.append(`${arg(field)}${idx !== fields.length - 2 ? ',' : ') {'}`);
             });
 
-            // Generate equals() method.
-            this.commentBlock(sb, '{@inheritDoc}');
-            sb.startBlock('@Override public boolean equals(Object o) {');
-            sb.startBlock('if (this == o)');
-            sb.append('return true;');
+            _.forEach(fields, (field) => sb.append(`this.${field.javaFieldName} = ${field.javaFieldName};`));
 
-            sb.endBlock('');
+            sb.endBlock('}');
 
-            sb.startBlock(`if (!(o instanceof ${clsName}))`);
-            sb.append('return false;');
+            sb.emptyLine();
+        }
 
-            sb.endBlock('');
+        // Generate getters and setters methods.
+        _.forEach(fields, (field) => {
+            const fldType = this.javaTypes.shortClassName(field.javaFieldType);
+            const fldName = field.javaFieldName;
 
-            sb.append(`${clsName} that = (${clsName})o;`);
+            this.commentBlock(sb,
+                `Gets ${fldName}`,
+                '',
+                `@return Value for ${fldName}.`
+            );
+            sb.startBlock(`public ${fldType} ${this.javaTypes.toJavaName('get', fldName)}() {`);
+            sb.append('return ' + fldName + ';');
+            sb.endBlock('}');
 
-            _.forEach(fields, (field) => {
-                sb.emptyLine();
+            sb.emptyLine();
 
-                const javaName = field.javaFieldName;
-                const javaType = field.javaFieldType;
+            this.commentBlock(sb,
+                `Sets ${fldName}`,
+                '',
+                `@param ${fldName} New value for ${fldName}.`
+            );
+            sb.startBlock(`public void ${this.javaTypes.toJavaName('set', fldName)}(${fldType} ${fldName}) {`);
+            sb.append(`this.${fldName} = ${fldName};`);
+            sb.endBlock('}');
 
-                switch (javaType) {
-                    case 'float':
-                        sb.startBlock(`if (Float.compare(${javaName}, that.${javaName}) != 0)`);
+            sb.emptyLine();
+        });
 
-                        break;
-                    case 'double':
-                        sb.startBlock(`if (Double.compare(${javaName}, that.${javaName}) != 0)`);
+        // Generate equals() method.
+        this.commentBlock(sb, '{@inheritDoc}');
+        sb.startBlock('@Override public boolean equals(Object o) {');
+        sb.startBlock('if (this == o)');
+        sb.append('return true;');
 
-                        break;
-                    default:
-                        if (JavaTypes.isJavaPrimitive(javaType))
-                            sb.startBlock('if (' + javaName + ' != that.' + javaName + ')');
-                        else
-                            sb.startBlock('if (' + javaName + ' != null ? !' + javaName + '.equals(that.' + javaName + ') : that.' + javaName + ' != null)');
-                }
+        sb.endBlock('');
 
-                sb.append('return false;');
+        sb.startBlock(`if (!(o instanceof ${clsName}))`);
+        sb.append('return false;');
 
-                sb.endBlock('');
-            });
+        sb.endBlock('');
 
-            sb.append('return true;');
-            sb.endBlock('}');
+        sb.append(`${clsName} that = (${clsName})o;`);
 
+        _.forEach(fields, (field) => {
             sb.emptyLine();
 
-            // Generate hashCode() method.
-            this.commentBlock(sb, '{@inheritDoc}');
-            sb.startBlock('@Override public int hashCode() {');
+            const javaName = field.javaFieldName;
+            const javaType = field.javaFieldType;
 
-            let first = true;
-            let tempVar = false;
+            switch (javaType) {
+                case 'float':
+                    sb.startBlock(`if (Float.compare(${javaName}, that.${javaName}) != 0)`);
 
-            _.forEach(fields, (field) => {
-                const javaName = field.javaFieldName;
-                const javaType = field.javaFieldType;
+                    break;
+                case 'double':
+                    sb.startBlock(`if (Double.compare(${javaName}, that.${javaName}) != 0)`);
 
-                let fldHashCode;
+                    break;
+                default:
+                    if (this.javaTypes.isPrimitive(javaType))
+                        sb.startBlock('if (' + javaName + ' != that.' + javaName + ')');
+                    else
+                        sb.startBlock('if (' + javaName + ' != null ? !' + javaName + '.equals(that.' + javaName + ') : that.' + javaName + ' != null)');
+            }
 
-                switch (javaType) {
-                    case 'boolean':
-                        fldHashCode = `${javaName} ? 1 : 0`;
+            sb.append('return false;');
 
-                        break;
-                    case 'byte':
-                    case 'short':
-                        fldHashCode = `(int)${javaName}`;
+            sb.endBlock('');
+        });
 
-                        break;
-                    case 'int':
-                        fldHashCode = `${javaName}`;
+        sb.append('return true;');
+        sb.endBlock('}');
 
-                        break;
-                    case 'long':
-                        fldHashCode = `(int)(${javaName} ^ (${javaName} >>> 32))`;
+        sb.emptyLine();
 
-                        break;
-                    case 'float':
-                        fldHashCode = `${javaName} != +0.0f ? Float.floatToIntBits(${javaName}) : 0`;
+        // Generate hashCode() method.
+        this.commentBlock(sb, '{@inheritDoc}');
+        sb.startBlock('@Override public int hashCode() {');
 
-                        break;
-                    case 'double':
-                        sb.append(`${tempVar ? 'ig_hash_temp' : 'long ig_hash_temp'} = Double.doubleToLongBits(${javaName});`);
+        let first = true;
+        let tempVar = false;
 
-                        tempVar = true;
+        _.forEach(fields, (field) => {
+            const javaName = field.javaFieldName;
+            const javaType = field.javaFieldType;
 
-                        fldHashCode = `${javaName} != +0.0f ? Float.floatToIntBits(${javaName}) : 0`;
+            let fldHashCode;
 
-                        break;
-                    default:
-                        fldHashCode = `${javaName} != null ? ${javaName}.hashCode() : 0`;
-                }
+            switch (javaType) {
+                case 'boolean':
+                    fldHashCode = `${javaName} ? 1 : 0`;
 
-                sb.append(first ? `int res = ${fldHashCode};` : `res = 31 * res + ${fldHashCode.startsWith('(') ? fldHashCode : `(${fldHashCode})`};`);
+                    break;
+                case 'byte':
+                case 'short':
+                    fldHashCode = `(int)${javaName}`;
 
-                first = false;
+                    break;
+                case 'int':
+                    fldHashCode = `${javaName}`;
 
-                sb.emptyLine();
-            });
+                    break;
+                case 'long':
+                    fldHashCode = `(int)(${javaName} ^ (${javaName} >>> 32))`;
 
-            sb.append('return res;');
-            sb.endBlock('}');
+                    break;
+                case 'float':
+                    fldHashCode = `${javaName} != +0.0f ? Float.floatToIntBits(${javaName}) : 0`;
 
-            sb.emptyLine();
+                    break;
+                case 'double':
+                    sb.append(`${tempVar ? 'ig_hash_temp' : 'long ig_hash_temp'} = Double.doubleToLongBits(${javaName});`);
 
-            this.commentBlock(sb, '{@inheritDoc}');
-            sb.startBlock('@Override public String toString() {');
-            sb.startBlock(`return "${clsName} [" + `);
+                    tempVar = true;
 
-            _.forEach(fields, (field, idx) => {
-                sb.append(`"${field.javaFieldName}=" + ${field.javaFieldName}${idx < fields.length - 1 ? ' + ", " + ' : ' +'}`);
-            });
+                    fldHashCode = `${javaName} != +0.0f ? Float.floatToIntBits(${javaName}) : 0`;
 
-            sb.endBlock('"]";');
-            sb.endBlock('}');
+                    break;
+                default:
+                    fldHashCode = `${javaName} != null ? ${javaName}.hashCode() : 0`;
+            }
 
-            sb.endBlock('}');
+            sb.append(first ? `int res = ${fldHashCode};` : `res = 31 * res + ${fldHashCode.startsWith('(') ? fldHashCode : `(${fldHashCode})`};`);
 
-            return sb.asString();
-        }
+            first = false;
 
-        /**
-         * Generate source code for type by its domain models.
-         *
-         * @param caches List of caches to generate POJOs for.
-         * @param addConstructor If 'true' then generate constructors.
-         * @param includeKeyFields If 'true' then include key fields into value POJO.
-         */
-        static pojos(caches, addConstructor, includeKeyFields) {
-            const pojos = [];
-
-            _.forEach(caches, (cache) => {
-                _.forEach(cache.domains, (domain) => {
-                    // Process only  domains with 'generatePojo' flag and skip already generated classes.
-                    if (domain.generatePojo && !_.find(pojos, {valueType: domain.valueType}) &&
-                        // Skip domain models without value fields.
-                        _.nonEmpty(domain.valueFields)) {
-                        const pojo = {};
-
-                        // Key class generation only if key is not build in java class.
-                        if (_.nonNil(domain.keyFields) && domain.keyFields.length > 0) {
-                            pojo.keyType = domain.keyType;
-                            pojo.keyClass = this.pojo(domain.keyType, domain.keyFields, addConstructor);
-                        }
+            sb.emptyLine();
+        });
 
-                        const valueFields = _.clone(domain.valueFields);
+        sb.append('return res;');
+        sb.endBlock('}');
 
-                        if (includeKeyFields) {
-                            _.forEach(domain.keyFields, (fld) => {
-                                if (!_.find(valueFields, {javaFieldName: fld.javaFieldName}))
-                                    valueFields.push(fld);
-                            });
-                        }
+        sb.emptyLine();
 
-                        pojo.valueType = domain.valueType;
-                        pojo.valueClass = this.pojo(domain.valueType, valueFields, addConstructor);
+        this.commentBlock(sb, '{@inheritDoc}');
+        sb.startBlock('@Override public String toString() {');
+        sb.startBlock(`return "${clsName} [" + `);
 
-                        pojos.push(pojo);
-                    }
-                });
-            });
+        _.forEach(fields, (field, idx) => {
+            sb.append(`"${field.javaFieldName}=" + ${field.javaFieldName}${idx < fields.length - 1 ? ' + ", " + ' : ' +'}`);
+        });
 
-            return pojos;
-        }
+        sb.endBlock('"]";');
+        sb.endBlock('}');
 
-        // Generate creation and execution of cache query.
-        static _multilineQuery(sb, query, prefix, postfix) {
-            if (_.isEmpty(query))
-                return;
+        sb.endBlock('}');
 
-            _.forEach(query, (line, ix) => {
-                if (ix === 0) {
-                    if (query.length === 1)
-                        sb.append(`${prefix}"${line}"${postfix}`);
-                    else
-                        sb.startBlock(`${prefix}"${line}" +`);
+        return sb.asString();
+    }
+
+    /**
+     * Generate source code for type by its domain models.
+     *
+     * @param caches List of caches to generate POJOs for.
+     * @param addConstructor If 'true' then generate constructors.
+     * @param includeKeyFields If 'true' then include key fields into value POJO.
+     */
+    static pojos(caches, addConstructor, includeKeyFields) {
+        const pojos = [];
+
+        _.forEach(caches, (cache) => {
+            _.forEach(cache.domains, (domain) => {
+                // Process only  domains with 'generatePojo' flag and skip already generated classes.
+                if (domain.generatePojo && !_.find(pojos, {valueType: domain.valueType}) &&
+                    // Skip domain models without value fields.
+                    _.nonEmpty(domain.valueFields)) {
+                    const pojo = {
+                        keyType: domain.keyType,
+                        valueType: domain.valueType
+                    };
+
+                    // Key class generation only if key is not build in java class.
+                    if (this.javaTypes.nonBuiltInClass(domain.keyType) && _.nonEmpty(domain.keyFields))
+                        pojo.keyClass = this.pojo(domain.keyType, domain.keyFields, addConstructor);
+
+                    const valueFields = _.clone(domain.valueFields);
+
+                    if (includeKeyFields) {
+                        _.forEach(domain.keyFields, (fld) => {
+                            if (!_.find(valueFields, {javaFieldName: fld.javaFieldName}))
+                                valueFields.push(fld);
+                        });
+                    }
+
+                    pojo.valueClass = this.pojo(domain.valueType, valueFields, addConstructor);
+
+                    pojos.push(pojo);
                 }
-                else
-                    sb.append(`"${line}"${ix === query.length - 1 ? postfix : ' +'}`);
             });
+        });
 
-            if (query.length > 1)
-                sb.endBlock('');
-            else
-                sb.emptyLine();
-        }
+        return pojos;
+    }
 
-        // Generate creation and execution of prepared statement.
-        static _prepareStatement(sb, conVar, query) {
-            this._multilineQuery(sb, query, `${conVar}.prepareStatement(`, ').executeUpdate();');
-        }
+    // Generate creation and execution of cache query.
+    static _multilineQuery(sb, query, prefix, postfix) {
+        if (_.isEmpty(query))
+            return;
 
-        static demoStartup(sb, cluster, shortFactoryCls) {
-            const cachesWithDataSource = _.filter(cluster.caches, (cache) => {
-                const kind = _.get(cache, 'cacheStoreFactory.kind');
+        _.forEach(query, (line, ix) => {
+            if (ix === 0) {
+                if (query.length === 1)
+                    sb.append(`${prefix}"${line}"${postfix}`);
+                else
+                    sb.startBlock(`${prefix}"${line}" +`);
+            }
+            else
+                sb.append(`"${line}"${ix === query.length - 1 ? postfix : ' +'}`);
+        });
 
-                if (kind) {
-                    const store = cache.cacheStoreFactory[kind];
+        if (query.length > 1)
+            sb.endBlock('');
+        else
+            sb.emptyLine();
+    }
 
-                    return (store.connectVia === 'DataSource' || _.isNil(store.connectVia)) && store.dialect;
-                }
+    // Generate creation and execution of prepared statement.
+    static _prepareStatement(sb, conVar, query) {
+        this._multilineQuery(sb, query, `${conVar}.prepareStatement(`, ').executeUpdate();');
+    }
 
-                return false;
-            });
+    static demoStartup(sb, cluster, shortFactoryCls) {
+        const cachesWithDataSource = _.filter(cluster.caches, (cache) => {
+            const kind = _.get(cache, 'cacheStoreFactory.kind');
 
-            const uniqDomains = [];
+            if (kind) {
+                const store = cache.cacheStoreFactory[kind];
 
-            // Prepare array of cache and his demo domain model list. Every domain is contained only in first cache.
-            const demoTypes = _.reduce(cachesWithDataSource, (acc, cache) => {
-                const domains = _.filter(cache.domains, (domain) => _.nonEmpty(domain.valueFields) &&
-                    !_.includes(uniqDomains, domain));
+                return (store.connectVia === 'DataSource' || _.isNil(store.connectVia)) && store.dialect;
+            }
 
-                if (_.nonEmpty(domains)) {
-                    uniqDomains.push(...domains);
+            return false;
+        });
 
-                    acc.push({
-                        cache,
-                        domains
-                    });
-                }
+        const uniqDomains = [];
 
-                return acc;
-            }, []);
+        // Prepare array of cache and his demo domain model list. Every domain is contained only in first cache.
+        const demoTypes = _.reduce(cachesWithDataSource, (acc, cache) => {
+            const domains = _.filter(cache.domains, (domain) => _.nonEmpty(domain.valueFields) &&
+                !_.includes(uniqDomains, domain));
 
-            if (_.nonEmpty(demoTypes)) {
-                // Group domain modes by data source
-                const typeByDs = _.groupBy(demoTypes, ({cache}) => cache.cacheStoreFactory[cache.cacheStoreFactory.kind].dataSourceBean);
+            if (_.nonEmpty(domains)) {
+                uniqDomains.push(...domains);
 
-                let rndNonDefined = true;
+                acc.push({
+                    cache,
+                    domains
+                });
+            }
 
-                const generatedConsts = [];
+            return acc;
+        }, []);
 
-                _.forEach(typeByDs, (types) => {
-                    _.forEach(types, (type) => {
-                        _.forEach(type.domains, (domain) => {
-                            const valType = domain.valueType.toUpperCase();
+        if (_.nonEmpty(demoTypes)) {
+            // Group domain modes by data source
+            const typeByDs = _.groupBy(demoTypes, ({cache}) => cache.cacheStoreFactory[cache.cacheStoreFactory.kind].dataSourceBean);
 
-                            const desc = _.find(PREDEFINED_QUERIES, (qry) => valType.endsWith(qry.type));
+            let rndNonDefined = true;
 
-                            if (desc) {
-                                if (rndNonDefined && desc.rndRequired) {
-                                    this.commentBlock(sb, 'Random generator for demo data.');
-                                    sb.append('private static final Random rnd = new Random();');
+            const generatedConsts = [];
 
-                                    sb.emptyLine();
+            _.forEach(typeByDs, (types) => {
+                _.forEach(types, (type) => {
+                    _.forEach(type.domains, (domain) => {
+                        const valType = domain.valueType.toUpperCase();
 
-                                    rndNonDefined = false;
-                                }
+                        const desc = _.find(PREDEFINED_QUERIES, (qry) => valType.endsWith(qry.type));
 
-                                _.forEach(desc.insertCntConsts, (cnt) => {
-                                    if (!_.includes(generatedConsts, cnt.name)) {
-                                        this.commentBlock(sb, cnt.comment);
-                                        sb.append(`private static final int ${cnt.name} = ${cnt.val};`);
+                        if (desc) {
+                            if (rndNonDefined && desc.rndRequired) {
+                                this.commentBlock(sb, 'Random generator for demo data.');
+                                sb.append('private static final Random rnd = new Random();');
 
-                                        sb.emptyLine();
+                                sb.emptyLine();
 
-                                        generatedConsts.push(cnt.name);
-                                    }
-                                });
+                                rndNonDefined = false;
                             }
-                        });
-                    });
-                });
 
-                // Generation of fill database method
-                this.commentBlock(sb, 'Fill data for Demo.');
-                sb.startBlock('private static void prepareDemoData() throws SQLException {');
+                            _.forEach(desc.insertCntConsts, (cnt) => {
+                                if (!_.includes(generatedConsts, cnt.name)) {
+                                    this.commentBlock(sb, cnt.comment);
+                                    sb.append(`private static final int ${cnt.name} = ${cnt.val};`);
 
-                let firstDs = true;
+                                    sb.emptyLine();
 
-                _.forEach(typeByDs, (types, ds) => {
-                    const conVar = ds + 'Con';
+                                    generatedConsts.push(cnt.name);
+                                }
+                            });
+                        }
+                    });
+                });
+            });
 
-                    if (firstDs)
-                        firstDs = false;
-                    else
-                        sb.emptyLine();
+            // Generation of fill database method
+            this.commentBlock(sb, 'Fill data for Demo.');
+            sb.startBlock('private static void prepareDemoData() throws SQLException {');
 
-                    sb.startBlock(`try (Connection ${conVar} = ${shortFactoryCls}.DataSources.INSTANCE_${ds}.getConnection()) {`);
+            let firstDs = true;
 
-                    let first = true;
-                    let stmtFirst = true;
+            _.forEach(typeByDs, (types, ds) => {
+                const conVar = ds + 'Con';
 
-                    _.forEach(types, (type) => {
-                        _.forEach(type.domains, (domain) => {
-                            const valType = domain.valueType.toUpperCase();
+                if (firstDs)
+                    firstDs = false;
+                else
+                    sb.emptyLine();
 
-                            const desc = _.find(PREDEFINED_QUERIES, (qry) => valType.endsWith(qry.type));
+                sb.startBlock(`try (Connection ${conVar} = ${shortFactoryCls}.DataSources.INSTANCE_${ds}.getConnection()) {`);
 
-                            if (desc) {
-                                if (first)
-                                    first = false;
-                                else
-                                    sb.emptyLine();
+                let first = true;
+                let stmtFirst = true;
 
-                                this.comment(sb, `Generate ${desc.type}.`);
+                _.forEach(types, (type) => {
+                    _.forEach(type.domains, (domain) => {
+                        const valType = domain.valueType.toUpperCase();
 
-                                if (desc.schema)
-                                    this._prepareStatement(sb, conVar, [`CREATE SCHEMA IF NOT EXISTS ${desc.schema}`]);
+                        const desc = _.find(PREDEFINED_QUERIES, (qry) => valType.endsWith(qry.type));
 
-                                this._prepareStatement(sb, conVar, desc.create);
+                        if (desc) {
+                            if (first)
+                                first = false;
+                            else
+                                sb.emptyLi

<TRUNCATED>

[30/50] [abbrv] ignite git commit: Web console beta-7.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js
new file mode 100644
index 0000000..2e01761
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js
@@ -0,0 +1,234 @@
+/*
+ * 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.
+ */
+
+import StringBuilder from './StringBuilder';
+
+// Java built-in class names.
+import POM_DEPENDENCIES from 'app/data/pom-dependencies.json';
+
+/**
+ * Pom file generation entry point.
+ */
+export default class IgniteMavenGenerator {
+    escapeId(s) {
+        if (typeof (s) !== 'string')
+            return s;
+
+        return s.replace(/[^A-Za-z0-9_\-.]+/g, '_');
+    }
+
+    addProperty(sb, tag, val) {
+        sb.append('<' + tag + '>' + val + '</' + tag + '>');
+    }
+
+    addDependency(deps, groupId, artifactId, version, jar) {
+        if (!_.find(deps, (dep) => dep.groupId === groupId && dep.artifactId === artifactId))
+            deps.push({groupId, artifactId, version, jar});
+    }
+
+    addResource(sb, dir, exclude) {
+        sb.startBlock('<resource>');
+        if (dir)
+            this.addProperty(sb, 'directory', dir);
+
+        if (exclude) {
+            sb.startBlock('<excludes>');
+            this.addProperty(sb, 'exclude', exclude);
+            sb.endBlock('</excludes>');
+        }
+
+        sb.endBlock('</resource>');
+    }
+
+    artifact(sb, cluster, version) {
+        this.addProperty(sb, 'groupId', 'org.apache.ignite');
+        this.addProperty(sb, 'artifactId', this.escapeId(cluster.name) + '-project');
+        this.addProperty(sb, 'version', version);
+
+        sb.emptyLine();
+    }
+
+    dependencies(sb, cluster, deps) {
+        sb.startBlock('<dependencies>');
+
+        _.forEach(deps, (dep) => {
+            sb.startBlock('<dependency>');
+
+            this.addProperty(sb, 'groupId', dep.groupId);
+            this.addProperty(sb, 'artifactId', dep.artifactId);
+            this.addProperty(sb, 'version', dep.version);
+
+            if (dep.jar) {
+                this.addProperty(sb, 'scope', 'system');
+                this.addProperty(sb, 'systemPath', '${project.basedir}/jdbc-drivers/' + dep.jar);
+            }
+
+            sb.endBlock('</dependency>');
+        });
+
+        sb.endBlock('</dependencies>');
+
+        return sb;
+    }
+
+    build(sb = new StringBuilder(), cluster, excludeGroupIds) {
+        sb.startBlock('<build>');
+        sb.startBlock('<resources>');
+        this.addResource(sb, 'src/main/java', '**/*.java');
+        this.addResource(sb, 'src/main/resources');
+        sb.endBlock('</resources>');
+
+        sb.startBlock('<plugins>');
+        sb.startBlock('<plugin>');
+        this.addProperty(sb, 'artifactId', 'maven-dependency-plugin');
+        sb.startBlock('<executions>');
+        sb.startBlock('<execution>');
+        this.addProperty(sb, 'id', 'copy-libs');
+        this.addProperty(sb, 'phase', 'test-compile');
+        sb.startBlock('<goals>');
+        this.addProperty(sb, 'goal', 'copy-dependencies');
+        sb.endBlock('</goals>');
+        sb.startBlock('<configuration>');
+        this.addProperty(sb, 'excludeGroupIds', excludeGroupIds.join(','));
+        this.addProperty(sb, 'outputDirectory', 'target/libs');
+        this.addProperty(sb, 'includeScope', 'compile');
+        this.addProperty(sb, 'excludeTransitive', 'true');
+        sb.endBlock('</configuration>');
+        sb.endBlock('</execution>');
+        sb.endBlock('</executions>');
+        sb.endBlock('</plugin>');
+        sb.startBlock('<plugin>');
+        this.addProperty(sb, 'artifactId', 'maven-compiler-plugin');
+        this.addProperty(sb, 'version', '3.1');
+        sb.startBlock('<configuration>');
+        this.addProperty(sb, 'source', '1.7');
+        this.addProperty(sb, 'target', '1.7');
+        sb.endBlock('</configuration>');
+        sb.endBlock('</plugin>');
+        sb.endBlock('</plugins>');
+        sb.endBlock('</build>');
+
+        sb.endBlock('</project>');
+    }
+
+    /**
+     * Add dependency for specified store factory if not exist.
+     * @param storeDeps Already added dependencies.
+     * @param storeFactory Store factory to add dependency.
+     */
+    storeFactoryDependency(storeDeps, storeFactory) {
+        if (storeFactory.dialect && (!storeFactory.connectVia || storeFactory.connectVia === 'DataSource')) {
+            const dep = POM_DEPENDENCIES[storeFactory.dialect];
+
+            this.addDependency(storeDeps, dep.groupId, dep.artifactId, dep.version, dep.jar);
+        }
+    }
+
+    /**
+     * Generate pom.xml.
+     *
+     * @param cluster Cluster  to take info about dependencies.
+     * @param version Ignite version for Ignite dependencies.
+     * @param sb Resulting output with generated pom.
+     * @returns {string} Generated content.
+     */
+    generate(cluster, version, sb = new StringBuilder()) {
+        const caches = cluster.caches;
+        const deps = [];
+        const storeDeps = [];
+        const excludeGroupIds = ['org.apache.ignite'];
+
+        const blobStoreFactory = {cacheStoreFactory: {kind: 'CacheHibernateBlobStoreFactory'}};
+
+        _.forEach(caches, (cache) => {
+            if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind)
+                this.storeFactoryDependency(storeDeps, cache.cacheStoreFactory[cache.cacheStoreFactory.kind]);
+
+            if (_.get(cache, 'nodeFilter.kind') === 'Exclude')
+                this.addDependency(deps, 'org.apache.ignite', 'ignite-extdata-p2p', version);
+        });
+
+        sb.append('<?xml version="1.0" encoding="UTF-8"?>');
+
+        sb.emptyLine();
+
+        sb.append(`<!-- ${sb.generatedBy()} -->`);
+
+        sb.emptyLine();
+
+        sb.startBlock('<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">');
+
+        sb.append('<modelVersion>4.0.0</modelVersion>');
+
+        sb.emptyLine();
+
+        this.artifact(sb, cluster, version);
+
+        this.addDependency(deps, 'org.apache.ignite', 'ignite-core', version);
+
+        this.addDependency(deps, 'org.apache.ignite', 'ignite-spring', version);
+        this.addDependency(deps, 'org.apache.ignite', 'ignite-indexing', version);
+        this.addDependency(deps, 'org.apache.ignite', 'ignite-rest-http', version);
+
+        if (_.get(cluster, 'deploymentSpi.kind') === 'URI')
+            this.addDependency(deps, 'org.apache.ignite', 'ignite-urideploy', version);
+
+        let dep = POM_DEPENDENCIES[cluster.discovery.kind];
+
+        if (dep)
+            this.addDependency(deps, 'org.apache.ignite', dep.artifactId, version);
+
+        if (cluster.discovery.kind === 'Jdbc') {
+            const store = cluster.discovery.Jdbc;
+
+            if (store.dataSourceBean && store.dialect)
+                this.storeFactoryDependency(storeDeps, cluster.discovery.Jdbc);
+        }
+
+        _.forEach(cluster.checkpointSpi, (spi) => {
+            if (spi.kind === 'S3') {
+                dep = POM_DEPENDENCIES.S3;
+
+                if (dep)
+                    this.addDependency(deps, 'org.apache.ignite', dep.artifactId, version);
+            }
+            else if (spi.kind === 'JDBC')
+                this.storeFactoryDependency(storeDeps, spi.JDBC);
+        });
+
+        if (_.find(cluster.igfss, (igfs) => igfs.secondaryFileSystemEnabled))
+            this.addDependency(deps, 'org.apache.ignite', 'ignite-hadoop', version);
+
+        if (_.find(caches, blobStoreFactory))
+            this.addDependency(deps, 'org.apache.ignite', 'ignite-hibernate', version);
+
+        if (cluster.logger && cluster.logger.kind) {
+            dep = POM_DEPENDENCIES[cluster.logger.kind];
+
+            if (dep)
+                this.addDependency(deps, 'org.apache.ignite', dep.artifactId, version);
+        }
+
+        this.dependencies(sb, cluster, deps.concat(storeDeps));
+
+        sb.emptyLine();
+
+        this.build(sb, cluster, excludeGroupIds);
+
+        return sb;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js
deleted file mode 100644
index db58532..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js
+++ /dev/null
@@ -1,233 +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.
- */
-
-import StringBuilder from './StringBuilder';
-
-// Java built-in class names.
-import POM_DEPENDENCIES from 'app/data/pom-dependencies.json';
-
-/**
- * Pom file generation entry point.
- */
-class GeneratorPom {
-    escapeId(s) {
-        if (typeof (s) !== 'string')
-            return s;
-
-        return s.replace(/[^A-Za-z0-9_\-.]+/g, '_');
-    }
-
-    addProperty(sb, tag, val) {
-        sb.append('<' + tag + '>' + val + '</' + tag + '>');
-    }
-
-    addDependency(deps, groupId, artifactId, version, jar) {
-        if (!_.find(deps, (dep) => dep.groupId === groupId && dep.artifactId === artifactId))
-            deps.push({groupId, artifactId, version, jar});
-    }
-
-    addResource(sb, dir, exclude) {
-        sb.startBlock('<resource>');
-        if (dir)
-            this.addProperty(sb, 'directory', dir);
-
-        if (exclude) {
-            sb.startBlock('<excludes>');
-            this.addProperty(sb, 'exclude', exclude);
-            sb.endBlock('</excludes>');
-        }
-
-        sb.endBlock('</resource>');
-    }
-
-    artifact(sb, cluster, version) {
-        this.addProperty(sb, 'groupId', 'org.apache.ignite');
-        this.addProperty(sb, 'artifactId', this.escapeId(cluster.name) + '-project');
-        this.addProperty(sb, 'version', version);
-
-        sb.emptyLine();
-    }
-
-    dependencies(sb, cluster, deps) {
-        sb.startBlock('<dependencies>');
-
-        _.forEach(deps, (dep) => {
-            sb.startBlock('<dependency>');
-
-            this.addProperty(sb, 'groupId', dep.groupId);
-            this.addProperty(sb, 'artifactId', dep.artifactId);
-            this.addProperty(sb, 'version', dep.version);
-
-            if (dep.jar) {
-                this.addProperty(sb, 'scope', 'system');
-                this.addProperty(sb, 'systemPath', '${project.basedir}/jdbc-drivers/' + dep.jar);
-            }
-
-            sb.endBlock('</dependency>');
-        });
-
-        sb.endBlock('</dependencies>');
-
-        return sb;
-    }
-
-    build(sb = new StringBuilder(), cluster, excludeGroupIds) {
-        sb.startBlock('<build>');
-        sb.startBlock('<resources>');
-        this.addResource(sb, 'src/main/java', '**/*.java');
-        this.addResource(sb, 'src/main/resources');
-        sb.endBlock('</resources>');
-
-        sb.startBlock('<plugins>');
-        sb.startBlock('<plugin>');
-        this.addProperty(sb, 'artifactId', 'maven-dependency-plugin');
-        sb.startBlock('<executions>');
-        sb.startBlock('<execution>');
-        this.addProperty(sb, 'id', 'copy-libs');
-        this.addProperty(sb, 'phase', 'test-compile');
-        sb.startBlock('<goals>');
-        this.addProperty(sb, 'goal', 'copy-dependencies');
-        sb.endBlock('</goals>');
-        sb.startBlock('<configuration>');
-        this.addProperty(sb, 'excludeGroupIds', excludeGroupIds.join(','));
-        this.addProperty(sb, 'outputDirectory', 'target/libs');
-        this.addProperty(sb, 'includeScope', 'compile');
-        this.addProperty(sb, 'excludeTransitive', 'true');
-        sb.endBlock('</configuration>');
-        sb.endBlock('</execution>');
-        sb.endBlock('</executions>');
-        sb.endBlock('</plugin>');
-        sb.startBlock('<plugin>');
-        this.addProperty(sb, 'artifactId', 'maven-compiler-plugin');
-        this.addProperty(sb, 'version', '3.1');
-        sb.startBlock('<configuration>');
-        this.addProperty(sb, 'source', '1.7');
-        this.addProperty(sb, 'target', '1.7');
-        sb.endBlock('</configuration>');
-        sb.endBlock('</plugin>');
-        sb.endBlock('</plugins>');
-        sb.endBlock('</build>');
-
-        sb.endBlock('</project>');
-    }
-
-    /**
-     * Add dependency for specified store factory if not exist.
-     * @param storeDeps Already added dependencies.
-     * @param storeFactory Store factory to add dependency.
-     */
-    storeFactoryDependency(storeDeps, storeFactory) {
-        if (storeFactory.dialect && (!storeFactory.connectVia || storeFactory.connectVia === 'DataSource')) {
-            const dep = POM_DEPENDENCIES[storeFactory.dialect];
-
-            this.addDependency(storeDeps, dep.groupId, dep.artifactId, dep.version, dep.jar);
-        }
-    }
-
-    /**
-     * Generate pom.xml.
-     *
-     * @param cluster Cluster  to take info about dependencies.
-     * @param version Ignite version for Ignite dependencies.
-     * @param sb Resulting output with generated pom.
-     * @returns {string} Generated content.
-     */
-    generate(cluster, version, sb = new StringBuilder()) {
-        const caches = cluster.caches;
-        const deps = [];
-        const storeDeps = [];
-        const excludeGroupIds = ['org.apache.ignite'];
-
-        const blobStoreFactory = {cacheStoreFactory: {kind: 'CacheHibernateBlobStoreFactory'}};
-
-        _.forEach(caches, (cache) => {
-            if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind)
-                this.storeFactoryDependency(storeDeps, cache.cacheStoreFactory[cache.cacheStoreFactory.kind]);
-
-            if (_.get(cache, 'nodeFilter.kind') === 'Exclude')
-                this.addDependency(deps, 'org.apache.ignite', 'ignite-extdata-p2p', version);
-        });
-
-        sb.append('<?xml version="1.0" encoding="UTF-8"?>');
-
-        sb.emptyLine();
-
-        sb.append(`<!-- ${sb.generatedBy()} -->`);
-
-        sb.emptyLine();
-
-        sb.startBlock('<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">');
-
-        sb.append('<modelVersion>4.0.0</modelVersion>');
-
-        sb.emptyLine();
-
-        this.artifact(sb, cluster, version);
-
-        this.addDependency(deps, 'org.apache.ignite', 'ignite-core', version);
-
-        this.addDependency(deps, 'org.apache.ignite', 'ignite-spring', version);
-        this.addDependency(deps, 'org.apache.ignite', 'ignite-indexing', version);
-        this.addDependency(deps, 'org.apache.ignite', 'ignite-rest-http', version);
-
-        let dep = POM_DEPENDENCIES[cluster.discovery.kind];
-
-        if (dep)
-            this.addDependency(deps, 'org.apache.ignite', dep.artifactId, version);
-
-        if (cluster.discovery.kind === 'Jdbc') {
-            const store = cluster.discovery.Jdbc;
-
-            if (store.dataSourceBean && store.dialect)
-                this.storeFactoryDependency(storeDeps, cluster.discovery.Jdbc);
-        }
-
-        _.forEach(cluster.checkpointSpi, (spi) => {
-            if (spi.kind === 'S3') {
-                dep = POM_DEPENDENCIES.S3;
-
-                if (dep)
-                    this.addDependency(deps, 'org.apache.ignite', dep.artifactId, version);
-            }
-            else if (spi.kind === 'JDBC')
-                this.storeFactoryDependency(storeDeps, spi.JDBC);
-        });
-
-        if (_.find(cluster.igfss, (igfs) => igfs.secondaryFileSystemEnabled))
-            this.addDependency(deps, 'org.apache.ignite', 'ignite-hadoop', version);
-
-        if (_.find(caches, blobStoreFactory))
-            this.addDependency(deps, 'org.apache.ignite', 'ignite-hibernate', version);
-
-        if (cluster.logger && cluster.logger.kind) {
-            dep = POM_DEPENDENCIES[cluster.logger.kind];
-
-            if (dep)
-                this.addDependency(deps, 'org.apache.ignite', dep.artifactId, version);
-        }
-
-        this.dependencies(sb, cluster, deps.concat(storeDeps));
-
-        sb.emptyLine();
-
-        this.build(sb, cluster, excludeGroupIds);
-
-        return sb;
-    }
-}
-
-export default ['GeneratorPom', GeneratorPom];

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js
index 49b4aa6..8a6a471 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js
@@ -20,7 +20,7 @@ import StringBuilder from './StringBuilder';
 /**
  * Properties generation entry point.
  */
-export default class PropertiesGenerator {
+export default class IgnitePropertiesGenerator {
     _collectProperties(bean) {
         const props = [];
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/Readme.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Readme.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Readme.service.js
index 7043807..0aa34ee 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/Readme.service.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/Readme.service.js
@@ -20,7 +20,7 @@ import StringBuilder from './StringBuilder';
 /**
  * Properties generation entry point.
  */
-export default class ReadmeGenerator {
+export default class IgniteReadmeGenerator {
     header(sb) {
         sb.append('Content of this folder was generated by Apache Ignite Web Console');
         sb.append('=================================================================');

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/SharpTransformer.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/SharpTransformer.service.js b/modules/web-console/frontend/app/modules/configuration/generator/SharpTransformer.service.js
index 19043f6..6e6bffe 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/SharpTransformer.service.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/SharpTransformer.service.js
@@ -19,225 +19,238 @@ import _ from 'lodash';
 import AbstractTransformer from './AbstractTransformer';
 import StringBuilder from './StringBuilder';
 
-export default ['JavaTypes', 'IgnitePlatformGenerator', (JavaTypes, generator) => {
-    return class SharpTransformer extends AbstractTransformer {
-        static generator = generator;
-
-        static commentBlock(sb, ...lines) {
-            _.forEach(lines, (line) => sb.append(`// ${line}`));
-        }
-
-        static doc(sb, ...lines) {
-            sb.append('/// <summary>');
-            _.forEach(lines, (line) => sb.append(`/// ${line}`));
-            sb.append('/// </summary>');
-        }
-
-        static mainComment(sb) {
-            return this.doc(sb, sb.generatedBy());
-        }
-
-        /**
-         *
-         * @param {Array.<String>} sb
-         * @param {Bean} bean
-         */
-        static _defineBean(sb, bean) {
-            const shortClsName = JavaTypes.shortClassName(bean.clsName);
-
-            sb.append(`var ${bean.id} = new ${shortClsName}();`);
-        }
-
-        /**
-         * @param {StringBuilder} sb
-         * @param {Bean} parent
-         * @param {Bean} propertyName
-         * @param {String|Bean} value
-         * @private
-         */
-        static _setProperty(sb, parent, propertyName, value) {
-            sb.append(`${parent.id}.${_.upperFirst(propertyName)} = ${value};`);
-        }
-
-        /**
-         *
-         * @param {StringBuilder} sb
-         * @param {Bean} parent
-         * @param {String} propertyName
-         * @param {Bean} bean
-         * @private
-         */
-        static _setBeanProperty(sb, parent, propertyName, bean) {
-            sb.append(`${parent.id}.${_.upperFirst(propertyName)} = ${bean.id};`);
-        }
-
-        static _toObject(clsName, val) {
-            const items = _.isArray(val) ? val : [val];
-
-            return _.map(items, (item, idx) => {
-                if (_.isNil(item))
-                    return 'null';
-
-                const shortClsName = JavaTypes.shortClassName(clsName);
-
-                switch (shortClsName) {
-                    // case 'byte':
-                    //     return `(byte) ${item}`;
-                    // case 'Serializable':
-                    case 'String':
-                        if (items.length > 1)
-                            return `"${item}"${idx !== items.length - 1 ? ' +' : ''}`;
-
-                        return `"${item}"`;
-                    // case 'Path':
-                    //     return `"${item.replace(/\\/g, '\\\\')}"`;
-                    // case 'Class':
-                    //     return `${this.shortClassName(item)}.class`;
-                    // case 'UUID':
-                    //     return `UUID.fromString("${item}")`;
-                    // case 'PropertyChar':
-                    //     return `props.getProperty("${item}").toCharArray()`;
-                    // case 'Property':
-                    //     return `props.getProperty("${item}")`;
-                    // case 'Bean':
-                    //     if (item.isComplex())
-                    //         return item.id;
+import ConfigurationGenerator from './ConfigurationGenerator';
+
+import ClusterDefaults from './defaults/Cluster.service';
+import CacheDefaults from './defaults/Cache.service';
+import IGFSDefaults from './defaults/IGFS.service';
+
+import JavaTypes from '../../../services/JavaTypes.service';
+
+const generator = new ConfigurationGenerator();
+
+const clusterDflts = new ClusterDefaults();
+const cacheDflts = new CacheDefaults();
+const igfsDflts = new IGFSDefaults();
+
+const javaTypes = new JavaTypes(clusterDflts, cacheDflts, igfsDflts);
+
+export default class SharpTransformer extends AbstractTransformer {
+    static generator = generator;
+
+    static commentBlock(sb, ...lines) {
+        _.forEach(lines, (line) => sb.append(`// ${line}`));
+    }
+
+    static doc(sb, ...lines) {
+        sb.append('/// <summary>');
+        _.forEach(lines, (line) => sb.append(`/// ${line}`));
+        sb.append('/// </summary>');
+    }
+
+    static mainComment(sb) {
+        return this.doc(sb, sb.generatedBy());
+    }
+
+    /**
+     *
+     * @param {Array.<String>} sb
+     * @param {Bean} bean
+     */
+    static _defineBean(sb, bean) {
+        const shortClsName = javaTypes.shortClassName(bean.clsName);
+
+        sb.append(`var ${bean.id} = new ${shortClsName}();`);
+    }
+
+    /**
+     * @param {StringBuilder} sb
+     * @param {Bean} parent
+     * @param {Bean} propertyName
+     * @param {String|Bean} value
+     * @private
+     */
+    static _setProperty(sb, parent, propertyName, value) {
+        sb.append(`${parent.id}.${_.upperFirst(propertyName)} = ${value};`);
+    }
+
+    /**
+     *
+     * @param {StringBuilder} sb
+     * @param {Bean} parent
+     * @param {String} propertyName
+     * @param {Bean} bean
+     * @private
+     */
+    static _setBeanProperty(sb, parent, propertyName, bean) {
+        sb.append(`${parent.id}.${_.upperFirst(propertyName)} = ${bean.id};`);
+    }
+
+    static _toObject(clsName, val) {
+        const items = _.isArray(val) ? val : [val];
+
+        return _.map(items, (item, idx) => {
+            if (_.isNil(item))
+                return 'null';
+
+            const shortClsName = javaTypes.shortClassName(clsName);
+
+            switch (shortClsName) {
+                // case 'byte':
+                //     return `(byte) ${item}`;
+                // case 'Serializable':
+                case 'String':
+                    if (items.length > 1)
+                        return `"${item}"${idx !== items.length - 1 ? ' +' : ''}`;
+
+                    return `"${item}"`;
+                // case 'Path':
+                //     return `"${item.replace(/\\/g, '\\\\')}"`;
+                // case 'Class':
+                //     return `${this.shortClassName(item)}.class`;
+                // case 'UUID':
+                //     return `UUID.fromString("${item}")`;
+                // case 'PropertyChar':
+                //     return `props.getProperty("${item}").toCharArray()`;
+                // case 'Property':
+                //     return `props.getProperty("${item}")`;
+                // case 'Bean':
+                //     if (item.isComplex())
+                //         return item.id;
+                //
+                //     return this._newBean(item);
+                default:
+                    if (javaTypes.nonEnum(shortClsName))
+                        return item;
+
+                    return `${shortClsName}.${item}`;
+            }
+        });
+    }
+
+    /**
+     *
+     * @param {StringBuilder} sb
+     * @param {Bean} bean
+     * @returns {Array}
+     */
+    static _setProperties(sb = new StringBuilder(), bean) {
+        _.forEach(bean.properties, (prop) => {
+            switch (prop.clsName) {
+                case 'ICollection':
+                    // const implClsName = JavaTypes.shortClassName(prop.implClsName);
+
+                    const colTypeClsName = javaTypes.shortClassName(prop.typeClsName);
+
+                    if (colTypeClsName === 'String') {
+                        const items = this._toObject(colTypeClsName, prop.items);
+
+                        sb.append(`${bean.id}.${_.upperFirst(prop.name)} = new {${items.join(', ')}};`);
+                    }
+                    // else {
+                    //     if (_.includes(vars, prop.id))
+                    //         sb.append(`${prop.id} = new ${implClsName}<>();`);
+                    //     else {
+                    //         vars.push(prop.id);
                     //
-                    //     return this._newBean(item);
-                    default:
-                        if (JavaTypes.nonEnum(shortClsName))
-                            return item;
-
-                        return `${shortClsName}.${item}`;
-                }
-            });
-        }
-
-        /**
-         *
-         * @param {StringBuilder} sb
-         * @param {Bean} bean
-         * @returns {Array}
-         */
-        static _setProperties(sb = new StringBuilder(), bean) {
-            _.forEach(bean.properties, (prop) => {
-                switch (prop.clsName) {
-                    case 'ICollection':
-                        // const implClsName = JavaTypes.shortClassName(prop.implClsName);
-
-                        const colTypeClsName = JavaTypes.shortClassName(prop.typeClsName);
-
-                        if (colTypeClsName === 'String') {
-                            const items = this._toObject(colTypeClsName, prop.items);
-
-                            sb.append(`${bean.id}.${_.upperFirst(prop.name)} = new {${items.join(', ')}};`);
-                        }
-                        // else {
-                        //     if (_.includes(vars, prop.id))
-                        //         sb.append(`${prop.id} = new ${implClsName}<>();`);
-                        //     else {
-                        //         vars.push(prop.id);
-                        //
-                        //         sb.append(`${clsName}<${colTypeClsName}> ${prop.id} = new ${implClsName}<>();`);
-                        //     }
-                        //
-                        //     sb.emptyLine();
-                        //
-                        //     if (nonBean) {
-                        //         const items = this._toObject(colTypeClsName, prop.items);
-                        //
-                        //         _.forEach(items, (item) => {
-                        //             sb.append(`${prop.id}.add("${item}");`);
-                        //
-                        //             sb.emptyLine();
-                        //         });
-                        //     }
-                        //     else {
-                        //         _.forEach(prop.items, (item) => {
-                        //             this.constructBean(sb, item, vars, limitLines);
-                        //
-                        //             sb.append(`${prop.id}.add(${item.id});`);
-                        //
-                        //             sb.emptyLine();
-                        //         });
-                        //
-                        //         this._setProperty(sb, bean.id, prop.name, prop.id);
-                        //     }
-                        // }
-
-                        break;
-
-                    case 'Bean':
-                        const nestedBean = prop.value;
-
-                        this._defineBean(sb, nestedBean);
-
-                        sb.emptyLine();
-
-                        this._setProperties(sb, nestedBean);
-
-                        sb.emptyLine();
-
-                        this._setBeanProperty(sb, bean, prop.name, nestedBean);
-
-                        break;
-                    default:
-                        this._setProperty(sb, bean, prop.name, this._toObject(prop.clsName, prop.value));
-                }
-            });
-
-            return sb;
-        }
-
-        /**
-         * Build Java startup class with configuration.
-         *
-         * @param {Bean} cfg
-         * @param pkg Package name.
-         * @param clsName Class name for generate factory class otherwise generate code snippet.
-         * @param clientNearCfg Optional near cache configuration for client node.
-         * @returns {String}
-         */
-        static toClassFile(cfg, pkg, clsName) {
-            const sb = new StringBuilder();
-
-            sb.startBlock(`namespace ${pkg}`, '{');
-
-            _.forEach(_.sortBy(cfg.collectClasses()), (cls) => sb.append(`using ${cls};`));
-            sb.emptyLine();
-
-
-            this.mainComment(sb);
-            sb.startBlock(`public class ${clsName}`, '{');
-
-            this.doc(sb, 'Configure grid.');
-            sb.startBlock('public static IgniteConfiguration CreateConfiguration()', '{');
-
-            this._defineBean(sb, cfg);
-
-            sb.emptyLine();
-
-            this._setProperties(sb, cfg);
+                    //         sb.append(`${clsName}<${colTypeClsName}> ${prop.id} = new ${implClsName}<>();`);
+                    //     }
+                    //
+                    //     sb.emptyLine();
+                    //
+                    //     if (nonBean) {
+                    //         const items = this._toObject(colTypeClsName, prop.items);
+                    //
+                    //         _.forEach(items, (item) => {
+                    //             sb.append(`${prop.id}.add("${item}");`);
+                    //
+                    //             sb.emptyLine();
+                    //         });
+                    //     }
+                    //     else {
+                    //         _.forEach(prop.items, (item) => {
+                    //             this.constructBean(sb, item, vars, limitLines);
+                    //
+                    //             sb.append(`${prop.id}.add(${item.id});`);
+                    //
+                    //             sb.emptyLine();
+                    //         });
+                    //
+                    //         this._setProperty(sb, bean.id, prop.name, prop.id);
+                    //     }
+                    // }
+
+                    break;
+
+                case 'Bean':
+                    const nestedBean = prop.value;
+
+                    this._defineBean(sb, nestedBean);
+
+                    sb.emptyLine();
+
+                    this._setProperties(sb, nestedBean);
+
+                    sb.emptyLine();
+
+                    this._setBeanProperty(sb, bean, prop.name, nestedBean);
+
+                    break;
+                default:
+                    this._setProperty(sb, bean, prop.name, this._toObject(prop.clsName, prop.value));
+            }
+        });
+
+        return sb;
+    }
+
+    /**
+     * Build Java startup class with configuration.
+     *
+     * @param {Bean} cfg
+     * @param pkg Package name.
+     * @param clsName Class name for generate factory class otherwise generate code snippet.
+     * @returns {String}
+     */
+    static toClassFile(cfg, pkg, clsName) {
+        const sb = new StringBuilder();
+
+        sb.startBlock(`namespace ${pkg}`, '{');
+
+        _.forEach(_.sortBy(cfg.collectClasses()), (cls) => sb.append(`using ${cls};`));
+        sb.emptyLine();
+
+
+        this.mainComment(sb);
+        sb.startBlock(`public class ${clsName}`, '{');
+
+        this.doc(sb, 'Configure grid.');
+        sb.startBlock('public static IgniteConfiguration CreateConfiguration()', '{');
+
+        this._defineBean(sb, cfg);
+
+        sb.emptyLine();
+
+        this._setProperties(sb, cfg);
 
-            sb.emptyLine();
+        sb.emptyLine();
 
-            sb.append(`return ${cfg.id};`);
+        sb.append(`return ${cfg.id};`);
 
-            sb.endBlock('}');
+        sb.endBlock('}');
 
-            sb.endBlock('}');
+        sb.endBlock('}');
 
-            sb.endBlock('}');
+        sb.endBlock('}');
 
-            return sb.asString();
-        }
+        return sb.asString();
+    }
 
-        static generateSection(bean) {
-            const sb = new StringBuilder();
+    static generateSection(bean) {
+        const sb = new StringBuilder();
 
-            this._setProperties(sb, bean);
+        this._setProperties(sb, bean);
 
-            return sb.asString();
-        }
-    };
-}];
+        return sb.asString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js b/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js
index 73df25e..b234575 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js
@@ -20,314 +20,311 @@ import _ from 'lodash';
 import AbstractTransformer from './AbstractTransformer';
 import StringBuilder from './StringBuilder';
 
-const escapeXml = (str) => {
-    return str.replace(/&/g, '&amp;')
-        .replace(/"/g, '&quot;')
-        .replace(/'/g, '&apos;')
-        .replace(/>/g, '&gt;')
-        .replace(/</g, '&lt;');
-};
-
-export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator', (JavaTypes, eventGroups, generator) => {
-    return class SpringTransformer extends AbstractTransformer {
-        static generator = generator;
-
-        static commentBlock(sb, ...lines) {
-            if (lines.length > 1) {
-                sb.append('<!--');
-
-                _.forEach(lines, (line) => sb.append(`  ${line}`));
-
-                sb.append('-->');
-            }
-            else
-                sb.append(`<!-- ${_.head(lines)} -->`);
+export default class IgniteSpringTransformer extends AbstractTransformer {
+    static escapeXml(str) {
+        return str.replace(/&/g, '&amp;')
+            .replace(/"/g, '&quot;')
+            .replace(/'/g, '&apos;')
+            .replace(/>/g, '&gt;')
+            .replace(/</g, '&lt;');
+    }
+
+    static commentBlock(sb, ...lines) {
+        if (lines.length > 1) {
+            sb.append('<!--');
+
+            _.forEach(lines, (line) => sb.append(`  ${line}`));
+
+            sb.append('-->');
         }
+        else
+            sb.append(`<!-- ${_.head(lines)} -->`);
+    }
 
-        static appendBean(sb, bean, appendId) {
-            const beanTags = [];
-
-            if (appendId)
-                beanTags.push(`id="${bean.id}"`);
-
-            beanTags.push(`class="${bean.clsName}"`);
-
-            if (bean.factoryMtd)
-                beanTags.push(`factory-method="${bean.factoryMtd}"`);
-
-            sb.startBlock(`<bean ${beanTags.join(' ')}>`);
-
-            _.forEach(bean.arguments, (arg) => {
-                if (arg.clsName === 'MAP') {
-                    sb.startBlock('<constructor-arg>');
-                    this._constructMap(sb, arg);
-                    sb.endBlock('</constructor-arg>');
-                }
-                else if (_.isNil(arg.value)) {
-                    sb.startBlock('<constructor-arg>');
-                    sb.append('<null/>');
-                    sb.endBlock('</constructor-arg>');
-                }
-                else if (arg.constant) {
-                    sb.startBlock('<constructor-arg>');
-                    sb.append(`<util:constant static-field="${arg.clsName}.${arg.value}"/>`);
-                    sb.endBlock('</constructor-arg>');
-                }
-                else if (arg.clsName === 'BEAN') {
-                    sb.startBlock('<constructor-arg>');
-                    this.appendBean(sb, arg.value);
-                    sb.endBlock('</constructor-arg>');
-                }
-                else
-                    sb.append(`<constructor-arg value="${this._toObject(arg.clsName, arg.value)}"/>`);
-            });
+    static appendBean(sb, bean, appendId) {
+        const beanTags = [];
 
-            this._setProperties(sb, bean);
+        if (appendId)
+            beanTags.push(`id="${bean.id}"`);
 
-            sb.endBlock('</bean>');
-        }
+        beanTags.push(`class="${bean.clsName}"`);
 
-        static _toObject(clsName, items) {
-            return _.map(_.isArray(items) ? items : [items], (item) => {
-                switch (clsName) {
-                    case 'PROPERTY':
-                    case 'PROPERTY_CHAR':
-                    case 'PROPERTY_INT':
-                        return `\${${item}}`;
-                    case 'java.lang.Class':
-                        return JavaTypes.fullClassName(item);
-                    case 'long':
-                        return `${item}L`;
-                    case 'java.lang.String':
-                        return escapeXml(item);
-                    default:
-                        return item;
-                }
-            });
-        }
+        if (bean.factoryMtd)
+            beanTags.push(`factory-method="${bean.factoryMtd}"`);
 
-        static _isBean(clsName) {
-            return JavaTypes.nonBuiltInClass(clsName) && JavaTypes.nonEnum(clsName) && _.includes(clsName, '.');
-        }
+        sb.startBlock(`<bean ${beanTags.join(' ')}>`);
 
-        static _setCollection(sb, prop) {
-            sb.startBlock(`<property name="${prop.name}">`);
-            sb.startBlock('<list>');
+        _.forEach(bean.arguments, (arg) => {
+            if (arg.clsName === 'MAP') {
+                sb.startBlock('<constructor-arg>');
+                this._constructMap(sb, arg);
+                sb.endBlock('</constructor-arg>');
+            }
+            else if (_.isNil(arg.value)) {
+                sb.startBlock('<constructor-arg>');
+                sb.append('<null/>');
+                sb.endBlock('</constructor-arg>');
+            }
+            else if (arg.constant) {
+                sb.startBlock('<constructor-arg>');
+                sb.append(`<util:constant static-field="${arg.clsName}.${arg.value}"/>`);
+                sb.endBlock('</constructor-arg>');
+            }
+            else if (arg.clsName === 'BEAN') {
+                sb.startBlock('<constructor-arg>');
+                this.appendBean(sb, arg.value);
+                sb.endBlock('</constructor-arg>');
+            }
+            else
+                sb.append(`<constructor-arg value="${this._toObject(arg.clsName, arg.value)}"/>`);
+        });
+
+        this._setProperties(sb, bean);
+
+        sb.endBlock('</bean>');
+    }
+
+    static _toObject(clsName, items) {
+        return _.map(_.isArray(items) ? items : [items], (item) => {
+            switch (clsName) {
+                case 'PROPERTY':
+                case 'PROPERTY_CHAR':
+                case 'PROPERTY_INT':
+                    return `\${${item}}`;
+                case 'java.lang.Class':
+                    return this.javaTypes.fullClassName(item);
+                case 'long':
+                    return `${item}L`;
+                case 'java.lang.String':
+                case 'PATH':
+                    return this.escapeXml(item);
+                default:
+                    return item;
+            }
+        });
+    }
 
-            _.forEach(prop.items, (item, idx) => {
-                if (this._isBean(prop.typeClsName)) {
-                    if (idx !== 0)
-                        sb.emptyLine();
+    static _isBean(clsName) {
+        return this.javaTypes.nonBuiltInClass(clsName) && this.javaTypes.nonEnum(clsName) && _.includes(clsName, '.');
+    }
 
-                    this.appendBean(sb, item);
-                }
-                else
-                    sb.append(`<value>${item}</value>`);
-            });
+    static _setCollection(sb, prop) {
+        sb.startBlock(`<property name="${prop.name}">`);
+        sb.startBlock('<list>');
 
-            sb.endBlock('</list>');
-            sb.endBlock('</property>');
-        }
+        _.forEach(prop.items, (item, idx) => {
+            if (this._isBean(prop.typeClsName)) {
+                if (idx !== 0)
+                    sb.emptyLine();
 
-        static _constructMap(sb, map) {
-            sb.startBlock('<map>');
+                this.appendBean(sb, item);
+            }
+            else
+                sb.append(`<value>${item}</value>`);
+        });
 
-            _.forEach(map.entries, (entry) => {
-                const key = entry[map.keyField];
-                const val = entry[map.valField];
+        sb.endBlock('</list>');
+        sb.endBlock('</property>');
+    }
 
-                const isKeyBean = this._isBean(map.keyClsName);
-                const isValBean = this._isBean(map.valClsName);
+    static _constructMap(sb, map) {
+        sb.startBlock('<map>');
 
+        _.forEach(map.entries, (entry) => {
+            const key = entry[map.keyField];
+            const val = entry[map.valField];
 
-                if (isKeyBean || isValBean) {
-                    sb.startBlock('<entry>');
+            const isKeyBean = this._isBean(map.keyClsName);
+            const isValBean = this._isBean(map.valClsName);
 
-                    sb.startBlock('<key>');
-                    if (isKeyBean)
-                        this.appendBean(sb, key);
-                    else
-                        sb.append(this._toObject(map.keyClsName, key));
-                    sb.endBlock('</key>');
 
-                    sb.startBlock('<value>');
-                    if (isValBean)
-                        this.appendBean(sb, val);
-                    else
-                        sb.append(this._toObject(map.valClsName, val));
-                    sb.endBlock('</value>');
+            if (isKeyBean || isValBean) {
+                sb.startBlock('<entry>');
 
-                    sb.endBlock('</entry>');
-                }
+                sb.startBlock('<key>');
+                if (isKeyBean)
+                    this.appendBean(sb, key);
                 else
-                    sb.append(`<entry key="${this._toObject(map.keyClsName, key)}" value="${this._toObject(map.valClsName, val)}"/>`);
-            });
+                    sb.append(this._toObject(map.keyClsName, key));
+                sb.endBlock('</key>');
 
-            sb.endBlock('</map>');
-        }
+                sb.startBlock('<value>');
+                if (isValBean)
+                    this.appendBean(sb, val);
+                else
+                    sb.append(this._toObject(map.valClsName, val));
+                sb.endBlock('</value>');
 
-        /**
-         *
-         * @param {StringBuilder} sb
-         * @param {Bean} bean
-         * @returns {StringBuilder}
-         */
-        static _setProperties(sb, bean) {
-            _.forEach(bean.properties, (prop, idx) => {
-                switch (prop.clsName) {
-                    case 'DATA_SOURCE':
-                        const valAttr = prop.name === 'dataSource' ? 'ref' : 'value';
+                sb.endBlock('</entry>');
+            }
+            else
+                sb.append(`<entry key="${this._toObject(map.keyClsName, key)}" value="${this._toObject(map.valClsName, val)}"/>`);
+        });
 
-                        sb.append(`<property name="${prop.name}" ${valAttr}="${prop.id}"/>`);
+        sb.endBlock('</map>');
+    }
 
-                        break;
-                    case 'EVENT_TYPES':
-                        sb.startBlock(`<property name="${prop.name}">`);
+    /**
+     *
+     * @param {StringBuilder} sb
+     * @param {Bean} bean
+     * @returns {StringBuilder}
+     */
+    static _setProperties(sb, bean) {
+        _.forEach(bean.properties, (prop, idx) => {
+            switch (prop.clsName) {
+                case 'DATA_SOURCE':
+                    const valAttr = prop.name === 'dataSource' ? 'ref' : 'value';
 
-                        if (prop.eventTypes.length === 1) {
-                            const evtGrp = _.find(eventGroups, {value: _.head(prop.eventTypes)});
+                    sb.append(`<property name="${prop.name}" ${valAttr}="${prop.id}"/>`);
 
-                            evtGrp && sb.append(`<util:constant static-field="${evtGrp.class}.${evtGrp.value}"/>`);
-                        }
-                        else {
-                            sb.startBlock('<list>');
+                    break;
+                case 'EVENT_TYPES':
+                    sb.startBlock(`<property name="${prop.name}">`);
 
-                            _.forEach(prop.eventTypes, (item, ix) => {
-                                ix > 0 && sb.emptyLine();
+                    if (prop.eventTypes.length === 1) {
+                        const evtGrp = _.find(this.eventGroups, {value: _.head(prop.eventTypes)});
 
-                                const evtGrp = _.find(eventGroups, {value: item});
+                        evtGrp && sb.append(`<util:constant static-field="${evtGrp.class}.${evtGrp.value}"/>`);
+                    }
+                    else {
+                        sb.startBlock('<list>');
 
-                                if (evtGrp) {
-                                    sb.append(`<!-- EventType.${item} -->`);
+                        _.forEach(prop.eventTypes, (item, ix) => {
+                            ix > 0 && sb.emptyLine();
 
-                                    _.forEach(evtGrp.events, (event) =>
-                                        sb.append(`<util:constant static-field="${evtGrp.class}.${event}"/>`));
-                                }
-                            });
+                            const evtGrp = _.find(this.eventGroups, {value: item});
 
-                            sb.endBlock('</list>');
-                        }
+                            if (evtGrp) {
+                                sb.append(`<!-- EventType.${item} -->`);
 
-                        sb.endBlock('</property>');
+                                _.forEach(evtGrp.events, (event) =>
+                                    sb.append(`<util:constant static-field="${evtGrp.class}.${event}"/>`));
+                            }
+                        });
 
-                        break;
-                    case 'ARRAY':
-                    case 'COLLECTION':
-                        this._setCollection(sb, prop);
+                        sb.endBlock('</list>');
+                    }
 
-                        break;
-                    case 'MAP':
-                        sb.startBlock(`<property name="${prop.name}">`);
+                    sb.endBlock('</property>');
 
-                        this._constructMap(sb, prop);
+                    break;
+                case 'ARRAY':
+                case 'COLLECTION':
+                    this._setCollection(sb, prop);
 
-                        sb.endBlock('</property>');
+                    break;
+                case 'MAP':
+                    sb.startBlock(`<property name="${prop.name}">`);
 
-                        break;
-                    case 'java.util.Properties':
-                        sb.startBlock(`<property name="${prop.name}">`);
-                        sb.startBlock('<props>');
+                    this._constructMap(sb, prop);
 
-                        _.forEach(prop.entries, (entry) => {
-                            sb.append(`<prop key="${entry.name}">${entry.value}</prop>`);
-                        });
+                    sb.endBlock('</property>');
 
-                        sb.endBlock('</props>');
-                        sb.endBlock('</property>');
+                    break;
+                case 'java.util.Properties':
+                    sb.startBlock(`<property name="${prop.name}">`);
+                    sb.startBlock('<props>');
 
-                        break;
-                    case 'BEAN':
-                        sb.startBlock(`<property name="${prop.name}">`);
+                    _.forEach(prop.entries, (entry) => {
+                        sb.append(`<prop key="${entry.name}">${entry.value}</prop>`);
+                    });
 
-                        this.appendBean(sb, prop.value);
+                    sb.endBlock('</props>');
+                    sb.endBlock('</property>');
 
-                        sb.endBlock('</property>');
+                    break;
+                case 'BEAN':
+                    sb.startBlock(`<property name="${prop.name}">`);
 
-                        break;
-                    default:
-                        sb.append(`<property name="${prop.name}" value="${this._toObject(prop.clsName, prop.value)}"/>`);
-                }
+                    this.appendBean(sb, prop.value);
 
-                this._emptyLineIfNeeded(sb, bean.properties, idx);
-            });
+                    sb.endBlock('</property>');
 
-            return sb;
-        }
+                    break;
+                default:
+                    sb.append(`<property name="${prop.name}" value="${this._toObject(prop.clsName, prop.value)}"/>`);
+            }
 
-        /**
-         * Build final XML.
-         *
-         * @param {Bean} cfg Ignite configuration.
-         * @param {Boolean} clientNearCaches
-         * @returns {StringBuilder}
-         */
-        static igniteConfiguration(cfg, clientNearCaches) {
-            const sb = new StringBuilder();
-
-            // 0. Add header.
-            sb.append('<?xml version="1.0" encoding="UTF-8"?>');
-            sb.emptyLine();
+            this._emptyLineIfNeeded(sb, bean.properties, idx);
+        });
+
+        return sb;
+    }
+
+    /**
+     * Build final XML.
+     *
+     * @param {Bean} cfg Ignite configuration.
+     * @param {Boolean} clientNearCaches
+     * @returns {StringBuilder}
+     */
+    static igniteConfiguration(cfg, clientNearCaches) {
+        const sb = new StringBuilder();
+
+        // 0. Add header.
+        sb.append('<?xml version="1.0" encoding="UTF-8"?>');
+        sb.emptyLine();
+
+        this.mainComment(sb);
+        sb.emptyLine();
+
+        // 1. Start beans section.
+        sb.startBlock([
+            '<beans xmlns="http://www.springframework.org/schema/beans"',
+            '       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"',
+            '       xmlns:util="http://www.springframework.org/schema/util"',
+            '       xsi:schemaLocation="http://www.springframework.org/schema/beans',
+            '                           http://www.springframework.org/schema/beans/spring-beans.xsd',
+            '                           http://www.springframework.org/schema/util',
+            '                           http://www.springframework.org/schema/util/spring-util.xsd">']);
+
+        // 2. Add external property file
+        if (this.hasProperties(cfg)) {
+            this.commentBlock(sb, 'Load external properties file.');
+
+            sb.startBlock('<bean id="placeholderConfig" class="org.springframework.beans.factory.config.PropertyPlaceholderConfigurer">');
+            sb.append('<property name="location" value="classpath:secret.properties"/>');
+            sb.endBlock('</bean>');
 
-            this.mainComment(sb);
             sb.emptyLine();
+        }
 
-            // 1. Start beans section.
-            sb.startBlock([
-                '<beans xmlns="http://www.springframework.org/schema/beans"',
-                '       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"',
-                '       xmlns:util="http://www.springframework.org/schema/util"',
-                '       xsi:schemaLocation="http://www.springframework.org/schema/beans',
-                '                           http://www.springframework.org/schema/beans/spring-beans.xsd',
-                '                           http://www.springframework.org/schema/util',
-                '                           http://www.springframework.org/schema/util/spring-util.xsd">']);
+        // 3. Add data sources.
+        const dataSources = this.collectDataSources(cfg);
 
-            // 2. Add external property file
-            if (this.hasProperties(cfg)) {
-                this.commentBlock(sb, 'Load external properties file.');
+        if (dataSources.length) {
+            this.commentBlock(sb, 'Data source beans will be initialized from external properties file.');
 
-                sb.startBlock('<bean id="placeholderConfig" class="org.springframework.beans.factory.config.PropertyPlaceholderConfigurer">');
-                sb.append('<property name="location" value="classpath:secret.properties"/>');
-                sb.endBlock('</bean>');
+            _.forEach(dataSources, (ds) => {
+                this.appendBean(sb, ds, true);
 
                 sb.emptyLine();
-            }
-
-            // 3. Add data sources.
-            const dataSources = this.collectDataSources(cfg);
-
-            if (dataSources.length) {
-                this.commentBlock(sb, 'Data source beans will be initialized from external properties file.');
-
-                _.forEach(dataSources, (ds) => {
-                    this.appendBean(sb, ds, true);
-
-                    sb.emptyLine();
-                });
-            }
+            });
+        }
 
-            _.forEach(clientNearCaches, (cache) => {
-                this.commentBlock(sb, 'Configuration of near cache for cache "' + cache.name + '"');
+        _.forEach(clientNearCaches, (cache) => {
+            this.commentBlock(sb, `Configuration of near cache for cache "${cache.name}"`);
 
-                this.appendBean(sb, generator.cacheNearClient(cache), true);
+            this.appendBean(sb, this.generator.cacheNearClient(cache), true);
 
-                sb.emptyLine();
-            });
+            sb.emptyLine();
+        });
 
-            // 3. Add main content.
-            this.appendBean(sb, cfg);
+        // 3. Add main content.
+        this.appendBean(sb, cfg);
 
-            // 4. Close beans section.
-            sb.endBlock('</beans>');
+        // 4. Close beans section.
+        sb.endBlock('</beans>');
 
-            return sb;
-        }
+        return sb;
+    }
 
-        static cluster(cluster, client) {
-            const cfg = generator.igniteConfiguration(cluster, client);
+    static cluster(cluster, client) {
+        const cfg = this.generator.igniteConfiguration(cluster, client);
 
-            const clientNearCaches = client ? _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled')) : [];
+        const clientNearCaches = client ? _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled')) : [];
 
-            return this.igniteConfiguration(cfg, clientNearCaches);
-        }
-    };
-}];
+        return this.igniteConfiguration(cfg, clientNearCaches);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.platform.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.platform.service.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.platform.service.js
new file mode 100644
index 0000000..eeac3a0
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.platform.service.js
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+import _ from 'lodash';
+
+const enumValueMapper = (val) => _.capitalize(val);
+
+const DFLT_CACHE = {
+    cacheMode: {
+        clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheMode',
+        mapper: enumValueMapper
+    },
+    atomicityMode: {
+        clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheAtomicityMode',
+        mapper: enumValueMapper
+    },
+    memoryMode: {
+        clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheMemoryMode',
+        value: 'ONHEAP_TIERED',
+        mapper: enumValueMapper
+    },
+    atomicWriteOrderMode: {
+        clsName: 'org.apache.ignite.cache.CacheAtomicWriteOrderMode',
+        mapper: enumValueMapper
+    },
+    writeSynchronizationMode: {
+        clsName: 'org.apache.ignite.cache.CacheWriteSynchronizationMode',
+        value: 'PRIMARY_SYNC',
+        mapper: enumValueMapper
+    },
+    rebalanceMode: {
+        clsName: 'org.apache.ignite.cache.CacheRebalanceMode',
+        value: 'ASYNC',
+        mapper: enumValueMapper
+    }
+};
+
+export default class IgniteCachePlatformDefaults {
+    constructor() {
+        Object.assign(this, DFLT_CACHE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.service.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.service.js
new file mode 100644
index 0000000..14b315f
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.service.js
@@ -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.
+ */
+
+const DFLT_CACHE = {
+    cacheMode: {
+        clsName: 'org.apache.ignite.cache.CacheMode'
+    },
+    atomicityMode: {
+        clsName: 'org.apache.ignite.cache.CacheAtomicityMode'
+    },
+    memoryMode: {
+        clsName: 'org.apache.ignite.cache.CacheMemoryMode',
+        value: 'ONHEAP_TIERED'
+    },
+    offHeapMaxMemory: -1,
+    startSize: 1500000,
+    swapEnabled: false,
+    sqlOnheapRowCacheSize: 10240,
+    longQueryWarningTimeout: 3000,
+    snapshotableIndex: false,
+    sqlEscapeAll: false,
+    storeKeepBinary: false,
+    loadPreviousValue: false,
+    cacheStoreFactory: {
+        CacheJdbcPojoStoreFactory: {
+            batchSize: 512,
+            maximumWriteAttempts: 2,
+            parallelLoadCacheMinimumThreshold: 512,
+            sqlEscapeAll: false
+        }
+    },
+    readThrough: false,
+    writeThrough: false,
+    writeBehindEnabled: false,
+    writeBehindBatchSize: 512,
+    writeBehindFlushSize: 10240,
+    writeBehindFlushFrequency: 5000,
+    writeBehindFlushThreadCount: 1,
+    maxConcurrentAsyncOperations: 500,
+    defaultLockTimeout: 0,
+    atomicWriteOrderMode: {
+        clsName: 'org.apache.ignite.cache.CacheAtomicWriteOrderMode'
+    },
+    writeSynchronizationMode: {
+        clsName: 'org.apache.ignite.cache.CacheWriteSynchronizationMode',
+        value: 'PRIMARY_SYNC'
+    },
+    rebalanceMode: {
+        clsName: 'org.apache.ignite.cache.CacheRebalanceMode',
+        value: 'ASYNC'
+    },
+    rebalanceThreadPoolSize: 1,
+    rebalanceBatchSize: 524288,
+    rebalanceBatchesPrefetchCount: 2,
+    rebalanceOrder: 0,
+    rebalanceDelay: 0,
+    rebalanceTimeout: 10000,
+    rebalanceThrottle: 0,
+    statisticsEnabled: false,
+    managementEnabled: false,
+    nearConfiguration: {
+        nearStartSize: 375000
+    },
+    clientNearConfiguration: {
+        nearStartSize: 375000
+    },
+    evictionPolicy: {
+        LRU: {
+            batchSize: 1,
+            maxSize: 100000
+        },
+        FIFO: {
+            batchSize: 1,
+            maxSize: 100000
+        },
+        SORTED: {
+            batchSize: 1,
+            maxSize: 100000
+        }
+    },
+    queryMetadata: 'Configuration',
+    fields: {
+        keyClsName: 'java.lang.String',
+        valClsName: 'java.lang.String',
+        valField: 'className',
+        entries: []
+    },
+    aliases: {
+        keyClsName: 'java.lang.String',
+        valClsName: 'java.lang.String',
+        keyField: 'field',
+        valField: 'alias',
+        entries: []
+    },
+    indexes: {
+        indexType: {
+            clsName: 'org.apache.ignite.cache.QueryIndexType'
+        },
+        fields: {
+            keyClsName: 'java.lang.String',
+            valClsName: 'java.lang.Boolean',
+            valField: 'direction',
+            entries: []
+        }
+    },
+    typeField: {
+        databaseFieldType: {
+            clsName: 'java.sql.Types'
+        }
+    }
+};
+
+export default class IgniteCacheDefaults {
+    constructor() {
+        Object.assign(this, DFLT_CACHE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.platform.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.platform.service.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.platform.service.js
new file mode 100644
index 0000000..b701951
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.platform.service.js
@@ -0,0 +1,43 @@
+/*
+ * 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.
+ */
+
+const enumValueMapper = (val) => _.capitalize(val);
+
+const DFLT_CLUSTER = {
+    atomics: {
+        cacheMode: {
+            clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheMode',
+            mapper: enumValueMapper
+        }
+    },
+    transactionConfiguration: {
+        defaultTxConcurrency: {
+            clsName: 'Apache.Ignite.Core.Transactions.TransactionConcurrency',
+            mapper: enumValueMapper
+        },
+        defaultTxIsolation: {
+            clsName: 'Apache.Ignite.Core.Transactions.TransactionIsolation',
+            mapper: enumValueMapper
+        }
+    }
+};
+
+export default class IgniteClusterPlatformDefaults {
+    constructor() {
+        Object.assign(this, DFLT_CLUSTER);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js
new file mode 100644
index 0000000..6333ef9
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js
@@ -0,0 +1,289 @@
+/*
+ * 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.
+ */
+
+const DFLT_CLUSTER = {
+    localHost: '0.0.0.0',
+    discovery: {
+        localPort: 47500,
+        localPortRange: 100,
+        socketTimeout: 5000,
+        ackTimeout: 5000,
+        maxAckTimeout: 600000,
+        networkTimeout: 5000,
+        joinTimeout: 0,
+        threadPriority: 10,
+        heartbeatFrequency: 2000,
+        maxMissedHeartbeats: 1,
+        maxMissedClientHeartbeats: 5,
+        topHistorySize: 1000,
+        reconnectCount: 10,
+        statisticsPrintFrequency: 0,
+        ipFinderCleanFrequency: 60000,
+        forceServerMode: false,
+        clientReconnectDisabled: false,
+        Multicast: {
+            multicastGroup: '228.1.2.4',
+            multicastPort: 47400,
+            responseWaitTime: 500,
+            addressRequestAttempts: 2,
+            localAddress: '0.0.0.0'
+        },
+        Jdbc: {
+            initSchema: false
+        },
+        SharedFs: {
+            path: 'disco/tcp'
+        },
+        ZooKeeper: {
+            basePath: '/services',
+            serviceName: 'ignite',
+            allowDuplicateRegistrations: false,
+            ExponentialBackoff: {
+                baseSleepTimeMs: 1000,
+                maxRetries: 10
+            },
+            BoundedExponentialBackoffRetry: {
+                baseSleepTimeMs: 1000,
+                maxSleepTimeMs: 2147483647,
+                maxRetries: 10
+            },
+            UntilElapsed: {
+                maxElapsedTimeMs: 60000,
+                sleepMsBetweenRetries: 1000
+            },
+            RetryNTimes: {
+                n: 10,
+                sleepMsBetweenRetries: 1000
+            },
+            OneTime: {
+                sleepMsBetweenRetry: 1000
+            },
+            Forever: {
+                retryIntervalMs: 1000
+            }
+        }
+    },
+    atomics: {
+        atomicSequenceReserveSize: 1000,
+        backups: 0,
+        cacheMode: {
+            clsName: 'org.apache.ignite.cache.CacheMode',
+            value: 'PARTITIONED'
+        }
+    },
+    binary: {
+        compactFooter: true,
+        typeConfigurations: {
+            enum: false
+        }
+    },
+    collision: {
+        kind: null,
+        JobStealing: {
+            activeJobsThreshold: 95,
+            waitJobsThreshold: 0,
+            messageExpireTime: 1000,
+            maximumStealingAttempts: 5,
+            stealingEnabled: true,
+            stealingAttributes: {
+                keyClsName: 'java.lang.String',
+                valClsName: 'java.io.Serializable',
+                items: []
+            }
+        },
+        PriorityQueue: {
+            priorityAttributeKey: 'grid.task.priority',
+            jobPriorityAttributeKey: 'grid.job.priority',
+            defaultPriority: 0,
+            starvationIncrement: 1,
+            starvationPreventionEnabled: true
+        }
+    },
+    communication: {
+        localPort: 47100,
+        localPortRange: 100,
+        sharedMemoryPort: 48100,
+        directBuffer: false,
+        directSendBuffer: false,
+        idleConnectionTimeout: 30000,
+        connectTimeout: 5000,
+        maxConnectTimeout: 600000,
+        reconnectCount: 10,
+        socketSendBuffer: 32768,
+        socketReceiveBuffer: 32768,
+        messageQueueLimit: 1024,
+        tcpNoDelay: true,
+        ackSendThreshold: 16,
+        unacknowledgedMessagesBufferSize: 0,
+        socketWriteTimeout: 2000
+    },
+    networkTimeout: 5000,
+    networkSendRetryDelay: 1000,
+    networkSendRetryCount: 3,
+    discoveryStartupDelay: 60000,
+    connector: {
+        port: 11211,
+        portRange: 100,
+        idleTimeout: 7000,
+        idleQueryCursorTimeout: 600000,
+        idleQueryCursorCheckFrequency: 60000,
+        receiveBufferSize: 32768,
+        sendBufferSize: 32768,
+        sendQueueLimit: 0,
+        directBuffer: false,
+        noDelay: true,
+        sslEnabled: false,
+        sslClientAuth: false
+    },
+    deploymentMode: {
+        clsName: 'org.apache.ignite.configuration.DeploymentMode',
+        value: 'SHARED'
+    },
+    peerClassLoadingEnabled: false,
+    peerClassLoadingMissedResourcesCacheSize: 100,
+    peerClassLoadingThreadPoolSize: 2,
+    failoverSpi: {
+        JobStealing: {
+            maximumFailoverAttempts: 5
+        },
+        Always: {
+            maximumFailoverAttempts: 5
+        }
+    },
+    logger: {
+        Log4j: {
+            level: {
+                clsName: 'org.apache.log4j.Level'
+            }
+        },
+        Log4j2: {
+            level: {
+                clsName: 'org.apache.logging.log4j.Level'
+            }
+        }
+    },
+    marshalLocalJobs: false,
+    marshallerCacheKeepAliveTime: 10000,
+    metricsHistorySize: 10000,
+    metricsLogFrequency: 60000,
+    metricsUpdateFrequency: 2000,
+    clockSyncSamples: 8,
+    clockSyncFrequency: 120000,
+    timeServerPortBase: 31100,
+    timeServerPortRange: 100,
+    transactionConfiguration: {
+        defaultTxConcurrency: {
+            clsName: 'org.apache.ignite.transactions.TransactionConcurrency',
+            value: 'PESSIMISTIC'
+        },
+        defaultTxIsolation: {
+            clsName: 'org.apache.ignite.transactions.TransactionIsolation',
+            value: 'REPEATABLE_READ'
+        },
+        defaultTxTimeout: 0,
+        pessimisticTxLogLinger: 10000
+    },
+    attributes: {
+        keyClsName: 'java.lang.String',
+        valClsName: 'java.lang.String',
+        items: []
+    },
+    odbcConfiguration: {
+        endpointAddress: '0.0.0.0:10800..10810',
+        maxOpenCursors: 128
+    },
+    eventStorage: {
+        Memory: {
+            expireCount: 10000
+        }
+    },
+    checkpointSpi: {
+        S3: {
+            bucketNameSuffix: 'default-bucket',
+            clientConfiguration: {
+                protocol: {
+                    clsName: 'com.amazonaws.Protocol',
+                    value: 'HTTPS'
+                },
+                maxConnections: 50,
+                retryPolicy: {
+                    retryCondition: {
+                        clsName: 'com.amazonaws.retry.PredefinedRetryPolicies'
+                    },
+                    backoffStrategy: {
+                        clsName: 'com.amazonaws.retry.PredefinedRetryPolicies'
+                    },
+                    maxErrorRetry: {
+                        clsName: 'com.amazonaws.retry.PredefinedRetryPolicies'
+                    }
+                },
+                maxErrorRetry: -1,
+                socketTimeout: 50000,
+                connectionTimeout: 50000,
+                requestTimeout: 0,
+                socketSendBufferSizeHints: 0,
+                connectionTTL: -1,
+                connectionMaxIdleMillis: 60000,
+                responseMetadataCacheSize: 50,
+                useReaper: true,
+                useGzip: false,
+                preemptiveBasicProxyAuth: false,
+                useTcpKeepAlive: false
+            }
+        },
+        JDBC: {
+            checkpointTableName: 'CHECKPOINTS',
+            keyFieldName: 'NAME',
+            keyFieldType: 'VARCHAR',
+            valueFieldName: 'VALUE',
+            valueFieldType: 'BLOB',
+            expireDateFieldName: 'EXPIRE_DATE',
+            expireDateFieldType: 'DATETIME',
+            numberOfRetries: 2
+        }
+    },
+    loadBalancingSpi: {
+        RoundRobin: {
+            perTask: false
+        },
+        Adaptive: {
+            loadProbe: {
+                Job: {
+                    useAverage: true
+                },
+                CPU: {
+                    useAverage: true,
+                    useProcessors: true,
+                    processorCoefficient: 1
+                },
+                ProcessingTime: {
+                    useAverage: true
+                }
+            }
+        },
+        WeightedRandom: {
+            nodeWeight: 10,
+            useWeights: false
+        }
+    }
+};
+
+export default class IgniteClusterDefaults {
+    constructor() {
+        Object.assign(this, DFLT_CLUSTER);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Event-groups.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Event-groups.service.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Event-groups.service.js
new file mode 100644
index 0000000..315da1f
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Event-groups.service.js
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+import _ from 'lodash';
+
+// Events groups.
+import EVENT_GROUPS from 'app/data/event-groups.json';
+
+export default class IgniteEventGroups {
+    constructor() {
+        return _.clone(EVENT_GROUPS);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/IGFS.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/IGFS.service.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/IGFS.service.js
new file mode 100644
index 0000000..985a56e
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/IGFS.service.js
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+const DFLT_IGFS = {
+    defaultMode: {
+        clsName: 'org.apache.ignite.igfs.IgfsMode',
+        value: 'DUAL_ASYNC'
+    },
+    secondaryFileSystem: {
+
+    },
+    ipcEndpointConfiguration: {
+        type: {
+            clsName: 'org.apache.ignite.igfs.IgfsIpcEndpointType'
+        },
+        host: '127.0.0.1',
+        port: 10500,
+        memorySize: 262144,
+        tokenDirectoryPath: 'ipc/shmem'
+    },
+    fragmentizerConcurrentFiles: 0,
+    fragmentizerThrottlingBlockLength: 16777216,
+    fragmentizerThrottlingDelay: 200,
+    dualModeMaxPendingPutsSize: 0,
+    dualModePutExecutorServiceShutdown: false,
+    blockSize: 65536,
+    streamBufferSize: 65536,
+    maxSpaceSize: 0,
+    maximumTaskRangeLength: 0,
+    managementPort: 11400,
+    perNodeBatchSize: 100,
+    perNodeParallelBatchCount: 8,
+    prefetchBlocks: 0,
+    sequentialReadsBeforePrefetch: 0,
+    trashPurgeTimeout: 1000,
+    colocateMetadata: true,
+    relaxedConsistency: true,
+    pathModes: {
+        keyClsName: 'java.lang.String',
+        keyField: 'path',
+        valClsName: 'org.apache.ignite.igfs.IgfsMode',
+        valField: 'mode'
+    }
+};
+
+export default class IgniteIGFSDefaults {
+    constructor() {
+        Object.assign(this, DFLT_IGFS);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.platform.provider.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.platform.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.platform.provider.js
deleted file mode 100644
index f06e11b..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.platform.provider.js
+++ /dev/null
@@ -1,60 +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.
- */
-
-import _ from 'lodash';
-
-const enumValueMapper = (val) => _.capitalize(val);
-
-const DFLT_CACHE = {
-    cacheMode: {
-        clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheMode',
-        mapper: enumValueMapper
-    },
-    atomicityMode: {
-        clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheAtomicityMode',
-        mapper: enumValueMapper
-    },
-    memoryMode: {
-        clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheMemoryMode',
-        value: 'ONHEAP_TIERED',
-        mapper: enumValueMapper
-    },
-    atomicWriteOrderMode: {
-        clsName: 'org.apache.ignite.cache.CacheAtomicWriteOrderMode',
-        mapper: enumValueMapper
-    },
-    writeSynchronizationMode: {
-        clsName: 'org.apache.ignite.cache.CacheWriteSynchronizationMode',
-        value: 'PRIMARY_SYNC',
-        mapper: enumValueMapper
-    },
-    rebalanceMode: {
-        clsName: 'org.apache.ignite.cache.CacheRebalanceMode',
-        value: 'ASYNC',
-        mapper: enumValueMapper
-    }
-};
-
-export default function() {
-    this.append = (dflts) => {
-        _.merge(DFLT_CACHE, dflts);
-    };
-
-    this.$get = ['igniteCacheDefaults', (cacheDefaults) => {
-        return _.merge({}, cacheDefaults, DFLT_CACHE);
-    }];
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js
deleted file mode 100644
index f50e493..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js
+++ /dev/null
@@ -1,137 +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.
- */
-
-import _ from 'lodash';
-
-const DFLT_CACHE = {
-    cacheMode: {
-        clsName: 'org.apache.ignite.cache.CacheMode'
-    },
-    atomicityMode: {
-        clsName: 'org.apache.ignite.cache.CacheAtomicityMode'
-    },
-    memoryMode: {
-        clsName: 'org.apache.ignite.cache.CacheMemoryMode',
-        value: 'ONHEAP_TIERED'
-    },
-    offHeapMaxMemory: -1,
-    startSize: 1500000,
-    swapEnabled: false,
-    sqlOnheapRowCacheSize: 10240,
-    longQueryWarningTimeout: 3000,
-    snapshotableIndex: false,
-    sqlEscapeAll: false,
-    storeKeepBinary: false,
-    loadPreviousValue: false,
-    cacheStoreFactory: {
-        CacheJdbcPojoStoreFactory: {
-            batchSize: 512,
-            maximumWriteAttempts: 2,
-            parallelLoadCacheMinimumThreshold: 512,
-            sqlEscapeAll: false
-        }
-    },
-    readThrough: false,
-    writeThrough: false,
-    writeBehindEnabled: false,
-    writeBehindBatchSize: 512,
-    writeBehindFlushSize: 10240,
-    writeBehindFlushFrequency: 5000,
-    writeBehindFlushThreadCount: 1,
-    maxConcurrentAsyncOperations: 500,
-    defaultLockTimeout: 0,
-    atomicWriteOrderMode: {
-        clsName: 'org.apache.ignite.cache.CacheAtomicWriteOrderMode'
-    },
-    writeSynchronizationMode: {
-        clsName: 'org.apache.ignite.cache.CacheWriteSynchronizationMode',
-        value: 'PRIMARY_SYNC'
-    },
-    rebalanceMode: {
-        clsName: 'org.apache.ignite.cache.CacheRebalanceMode',
-        value: 'ASYNC'
-    },
-    rebalanceThreadPoolSize: 1,
-    rebalanceBatchSize: 524288,
-    rebalanceBatchesPrefetchCount: 2,
-    rebalanceOrder: 0,
-    rebalanceDelay: 0,
-    rebalanceTimeout: 10000,
-    rebalanceThrottle: 0,
-    statisticsEnabled: false,
-    managementEnabled: false,
-    nearConfiguration: {
-        nearStartSize: 375000
-    },
-    clientNearConfiguration: {
-        nearStartSize: 375000
-    },
-    evictionPolicy: {
-        LRU: {
-            batchSize: 1,
-            maxSize: 100000
-        },
-        FIFO: {
-            batchSize: 1,
-            maxSize: 100000
-        },
-        SORTED: {
-            batchSize: 1,
-            maxSize: 100000
-        }
-    },
-    queryMetadata: 'Configuration',
-    fields: {
-        keyClsName: 'java.lang.String',
-        valClsName: 'java.lang.String',
-        valField: 'className',
-        entries: []
-    },
-    aliases: {
-        keyClsName: 'java.lang.String',
-        valClsName: 'java.lang.String',
-        keyField: 'field',
-        valField: 'alias',
-        entries: []
-    },
-    indexes: {
-        indexType: {
-            clsName: 'org.apache.ignite.cache.QueryIndexType'
-        },
-        fields: {
-            keyClsName: 'java.lang.String',
-            valClsName: 'java.lang.Boolean',
-            valField: 'direction',
-            entries: []
-        }
-    },
-    typeField: {
-        databaseFieldType: {
-            clsName: 'java.sql.Types'
-        }
-    }
-};
-
-export default function() {
-    this.append = (dflts) => {
-        _.merge(DFLT_CACHE, dflts);
-    };
-
-    this.$get = [() => {
-        return DFLT_CACHE;
-    }];
-}


[40/50] [abbrv] ignite git commit: IGNITE-4385 .NET: Allow inline AsCacheQueryable in LINQ

Posted by yz...@apache.org.
IGNITE-4385 .NET: Allow inline AsCacheQueryable in LINQ


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

Branch: refs/heads/ignite-comm-balance-master
Commit: c040c3767b3962e125aac81cb0a6c25fa565cec4
Parents: 781e5b7
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Dec 23 19:07:17 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Dec 23 19:07:17 2016 +0300

----------------------------------------------------------------------
 .../Cache/Query/CacheLinqTest.cs                | 51 ++++++++++++++++++--
 .../Apache.Ignite.Linq/Impl/ExpressionWalker.cs |  8 +++
 2 files changed, 56 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c040c376/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
index e74f09f..798e7e8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
@@ -425,6 +425,19 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         }
 
         /// <summary>
+        /// Tests the cross cache join.
+        /// </summary>
+        [Test]
+        public void TestCrossCacheJoinInline()
+        {
+            var res = GetPersonCache().AsCacheQueryable().Join(GetRoleCache().AsCacheQueryable(), 
+                person => person.Key, role => role.Key.Foo, (person, role) => role).ToArray();
+
+            Assert.AreEqual(RoleCount, res.Length);
+            Assert.AreEqual(101, res[0].Key.Bar);
+        }
+
+        /// <summary>
         /// Tests the multi cache join.
         /// </summary>
         [Test]
@@ -516,7 +529,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         }
 
         /// <summary>
-        /// Tests the multiple from.
+        /// Tests query with multiple from clause.
         /// </summary>
         [Test]
         public void TestMultipleFrom()
@@ -539,6 +552,23 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         }
 
         /// <summary>
+        /// Tests query with multiple from clause with inline query sources.
+        /// </summary>
+        [Test]
+        public void TestMultipleFromInline()
+        {
+            var filtered =
+                from person in GetPersonCache().AsCacheQueryable()
+                from role in GetRoleCache().AsCacheQueryable()
+                where person.Key == role.Key.Foo
+                select new {Person = person.Value.Name, Role = role.Value.Name};
+
+            var res = filtered.ToArray();
+
+            Assert.AreEqual(RoleCount, res.Length);
+        }
+
+        /// <summary>
         /// Tests the join of a table to itself.
         /// </summary>
         [Test]
@@ -557,6 +587,18 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         }
 
         /// <summary>
+        /// Tests the join of a table to itself with inline queryable.
+        /// </summary>
+        [Test]
+        public void TestSelfJoinInline()
+        {
+            var qry = GetPersonCache().AsCacheQueryable().Join(GetPersonCache().AsCacheQueryable(), 
+                x => x.Value.Age, x => x.Key, (x, y) => x.Key);
+
+            Assert.AreEqual(PersonCount, qry.ToArray().Distinct().Count());
+        }
+
+        /// <summary>
         /// Tests the group by.
         /// </summary>
         [Test]
@@ -729,9 +771,11 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             var persons = GetPersonCache().AsCacheQueryable();
 
             // Invalid dateTime
-            var now = DateTime.Now;
             // ReSharper disable once ReturnValueOfPureMethodIsNotUsed
-            Assert.Throws<InvalidOperationException>(() => roles.Where(x => x.Value.Date > now).ToArray());
+            var ex = Assert.Throws<InvalidOperationException>(() =>
+                roles.Where(x => x.Value.Date > DateTime.Now).ToArray());
+            Assert.AreEqual("DateTime is not UTC. Only UTC DateTime can be used for interop with other platforms.", 
+                ex.Message);
 
             // Test retrieval
             var dates = roles.OrderBy(x => x.Value.Date).Select(x => x.Value.Date);
@@ -741,6 +785,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             // Filtering
             Assert.AreEqual(1, persons.Count(x => x.Value.Birthday == StartDateTime));
             Assert.AreEqual(PersonCount, persons.Count(x => x.Value.Birthday >= StartDateTime));
+            Assert.Greater(persons.Count(x => x.Value.Birthday > DateTime.UtcNow), 1);
 
             // Joins
             var join = 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c040c376/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs
index a447bf5..4407f96 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs
@@ -99,6 +99,14 @@ namespace Apache.Ignite.Linq.Impl
                     return queryable;
             }
 
+            var callExpr = expression as MethodCallExpression;
+
+            if (callExpr != null)
+            {
+                // This is usually a nested query with a call to AsCacheQueryable().
+                return (ICacheQueryableInternal) Expression.Lambda(callExpr).Compile().DynamicInvoke();
+            }
+
             if (throwWhenNotFound)
                 throw new NotSupportedException("Unexpected query source: " + expression);
 


[44/50] [abbrv] ignite git commit: IGNITE-2793 Added support for byte arrays to ODBC

Posted by yz...@apache.org.
IGNITE-2793 Added support for byte arrays to ODBC

This closes #1371


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 32d2629e1a8f828b0a34e363e793a5add2639f2e
Parents: 300750f
Author: Sergey Kalashnikov <sk...@gridgain.com>
Authored: Tue Dec 27 13:08:34 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Dec 27 13:08:34 2016 +0300

----------------------------------------------------------------------
 modules/platforms/cpp/binary/Makefile.am        |   2 +-
 .../ignite/impl/binary/binary_type_manager.h    |   2 +-
 .../src/impl/binary/binary_type_manager.cpp     |   6 +-
 .../binary/src/impl/interop/interop_memory.cpp  |   2 +-
 modules/platforms/cpp/common/Makefile.am        |   2 +-
 .../include/ignite/common/dynamic_size_array.h  |   2 +-
 modules/platforms/cpp/core-test/Makefile.am     |   2 +-
 .../cpp/core-test/src/cache_query_test.cpp      |   8 +-
 .../cpp/core-test/src/reference_test.cpp        |  14 +-
 modules/platforms/cpp/core/Makefile.am          |   2 +-
 .../cache/query/continuous/continuous_query.h   |   8 +-
 .../include/ignite/cache/query/query_cursor.h   |   4 +-
 .../query/continuous/continuous_query_impl.h    |   8 +-
 .../core/include/ignite/impl/handle_registry.h  |   2 +-
 .../cpp/core/include/ignite/impl/operations.h   |   4 +-
 .../cpp/core/src/impl/cache/cache_impl.cpp      |   5 +-
 .../cpp/core/src/impl/handle_registry.cpp       |   2 +-
 .../cpp/examples/odbc-example/Makefile.am       |   2 +-
 .../cpp/examples/putget-example/Makefile.am     |   2 +-
 .../cpp/examples/query-example/Makefile.am      |   2 +-
 modules/platforms/cpp/ignite/Makefile.am        |   2 +-
 modules/platforms/cpp/jni/Makefile.am           |   2 +-
 modules/platforms/cpp/jni/src/java.cpp          |   2 +-
 modules/platforms/cpp/odbc-test/Makefile.am     |   2 +-
 .../cpp/odbc-test/config/queries-default.xml    |   1 +
 .../odbc-test/include/sql_test_suite_fixture.h  |   3 +
 .../platforms/cpp/odbc-test/include/test_type.h |  20 +-
 .../platforms/cpp/odbc-test/src/column_test.cpp | 189 +++++++++++++++++++
 .../odbc-test/src/sql_test_suite_fixture.cpp    |  17 ++
 .../cpp/odbc-test/src/sql_types_test.cpp        | 131 +++++++++++++
 modules/platforms/cpp/odbc/Makefile.am          |   2 +-
 .../cpp/odbc/include/ignite/odbc/utility.h      |   9 +-
 modules/platforms/cpp/odbc/src/column.cpp       |  28 ++-
 .../cpp/odbc/src/config/connection_info.cpp     |   6 +-
 modules/platforms/cpp/odbc/src/connection.cpp   |   2 +
 modules/platforms/cpp/odbc/src/odbc.cpp         |   5 +-
 modules/platforms/cpp/odbc/src/utility.cpp      |  15 ++
 37 files changed, 469 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/binary/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/Makefile.am b/modules/platforms/cpp/binary/Makefile.am
index ca301a6..5ffc4fd 100644
--- a/modules/platforms/cpp/binary/Makefile.am
+++ b/modules/platforms/cpp/binary/Makefile.am
@@ -32,7 +32,7 @@ AM_CPPFLAGS = \
 
 AM_CXXFLAGS = \
     -Wall \
-    -std=c++0x
+    -std=c++03
 
 libignite_binary_la_LIBADD = \
     @top_srcdir@/common/libignite-common.la

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/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 8aca1a0..63853f8 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
@@ -88,7 +88,7 @@ namespace ignite
 
             private:
                 /** Current snapshots. */
-                ignite::common::concurrent::SharedPointer<std::map<int32_t, SPSnap>> snapshots;
+                ignite::common::concurrent::SharedPointer<std::map<int32_t, SPSnap> > snapshots;
 
                 /** Pending snapshots. */
                 std::vector<SPSnap>* pending;

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/binary/src/impl/binary/binary_type_manager.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/src/impl/binary/binary_type_manager.cpp b/modules/platforms/cpp/binary/src/impl/binary/binary_type_manager.cpp
index 9bd115c..8b5dfc1 100644
--- a/modules/platforms/cpp/binary/src/impl/binary/binary_type_manager.cpp
+++ b/modules/platforms/cpp/binary/src/impl/binary/binary_type_manager.cpp
@@ -28,7 +28,7 @@ namespace ignite
         namespace binary
         {
             BinaryTypeManager::BinaryTypeManager() : 
-                snapshots(SharedPointer<std::map<int32_t, SPSnap>>(new std::map<int32_t, SPSnap>)),
+                snapshots(SharedPointer<std::map<int32_t, SPSnap> >(new std::map<int32_t, SPSnap>)),
                 pending(new std::vector<SPSnap>()), 
                 cs(new CriticalSection()), 
                 pendingVer(0), ver(0)
@@ -46,7 +46,7 @@ namespace ignite
 
             SharedPointer<BinaryTypeHandler> BinaryTypeManager::GetHandler(int32_t typeId)
             {
-                SharedPointer<std::map<int32_t, SPSnap>> snapshots0 = snapshots;
+                SharedPointer<std::map<int32_t, SPSnap> > snapshots0 = snapshots;
 
                 SPSnap snapshot = (*snapshots0.Get())[typeId];
 
@@ -156,7 +156,7 @@ namespace ignite
                         if (!snapshotFound)
                             (*newSnapshots)[pendingSnap->GetTypeId()] = *it;
 
-                        snapshots = SharedPointer<std::map<int32_t, SPSnap>>(newSnapshots);
+                        snapshots = SharedPointer<std::map<int32_t, SPSnap> >(newSnapshots);
                     }
                     else
                     {

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/binary/src/impl/interop/interop_memory.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/src/impl/interop/interop_memory.cpp b/modules/platforms/cpp/binary/src/impl/interop/interop_memory.cpp
index 926b7fb..04b27ae 100644
--- a/modules/platforms/cpp/binary/src/impl/interop/interop_memory.cpp
+++ b/modules/platforms/cpp/binary/src/impl/interop/interop_memory.cpp
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
+#include <cstdlib>
 #include <ignite/ignite_error.h>
 
 #include "ignite/impl/interop/interop_memory.h"

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/common/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/Makefile.am b/modules/platforms/cpp/common/Makefile.am
index 2d16c2d..9c27a57 100644
--- a/modules/platforms/cpp/common/Makefile.am
+++ b/modules/platforms/cpp/common/Makefile.am
@@ -32,7 +32,7 @@ AM_CPPFLAGS = \
 
 AM_CXXFLAGS = \
     -Wall \
-    -std=c++0x
+    -std=c++03
 
 libignite_common_la_LDFLAGS = \
     -no-undefined \

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/common/include/ignite/common/dynamic_size_array.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/dynamic_size_array.h b/modules/platforms/cpp/common/include/ignite/common/dynamic_size_array.h
index a6bcc13..d233e44 100644
--- a/modules/platforms/cpp/common/include/ignite/common/dynamic_size_array.h
+++ b/modules/platforms/cpp/common/include/ignite/common/dynamic_size_array.h
@@ -36,7 +36,7 @@ namespace ignite
          * This is the analogue of the standard vector. It is needed to be used
          * in exported classes as we can't export standard library classes.
          */
-        template<typename T, typename A = DefaultAllocator<T>>
+        template<typename T, typename A = DefaultAllocator<T> >
         class IGNITE_IMPORT_EXPORT DynamicSizeArray
         {
         public:

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/core-test/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/Makefile.am b/modules/platforms/cpp/core-test/Makefile.am
index 3cf1d0e..a5a60ed 100644
--- a/modules/platforms/cpp/core-test/Makefile.am
+++ b/modules/platforms/cpp/core-test/Makefile.am
@@ -39,7 +39,7 @@ AM_CPPFLAGS = \
 
 AM_CXXFLAGS = \
     -Wall \
-    -std=c++0x
+    -std=c++03
 
 ignite_tests_LDADD = \
     @top_srcdir@/core/libignite.la \

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/core-test/src/cache_query_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/cache_query_test.cpp b/modules/platforms/cpp/core-test/src/cache_query_test.cpp
index 928d29e..b5bb170 100644
--- a/modules/platforms/cpp/core-test/src/cache_query_test.cpp
+++ b/modules/platforms/cpp/core-test/src/cache_query_test.cpp
@@ -362,7 +362,7 @@ void CheckGetAllFail(QueryCursor<int, QueryPerson>& cur)
 {
     try 
     {
-        std::vector<CacheEntry<int, QueryPerson>> res;
+        std::vector<CacheEntry<int, QueryPerson> > res;
 
         cur.GetAll(res);
 
@@ -406,7 +406,7 @@ void CheckEmpty(QueryFieldsCursor& cur)
  */
 void CheckEmptyGetAll(QueryCursor<int, QueryPerson>& cur)
 {
-    std::vector<CacheEntry<int, QueryPerson>> res;
+    std::vector<CacheEntry<int, QueryPerson> > res;
 
     cur.GetAll(res);
 
@@ -478,7 +478,7 @@ void CheckSingle(QueryFieldsCursor& cur, int key, const std::string& name, int a
  */
 void CheckSingleGetAll(QueryCursor<int, QueryPerson>& cur, int key, const std::string& name, int age)
 {
-    std::vector<CacheEntry<int, QueryPerson>> res;
+    std::vector<CacheEntry<int, QueryPerson> > res;
 
     cur.GetAll(res);
 
@@ -555,7 +555,7 @@ void CheckMultiple(QueryCursor<int, QueryPerson>& cur, int key1, const std::stri
 void CheckMultipleGetAll(QueryCursor<int, QueryPerson>& cur, int key1, const std::string& name1,
     int age1, int key2, const std::string& name2, int age2)
 {
-    std::vector<CacheEntry<int, QueryPerson>> res;
+    std::vector<CacheEntry<int, QueryPerson> > res;
 
     cur.GetAll(res);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/core-test/src/reference_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/reference_test.cpp b/modules/platforms/cpp/core-test/src/reference_test.cpp
index f5c3e8d..b240e2e 100644
--- a/modules/platforms/cpp/core-test/src/reference_test.cpp
+++ b/modules/platforms/cpp/core-test/src/reference_test.cpp
@@ -148,8 +148,10 @@ void TestFunctionConst3(ConstReference<C3> c3, int expected)
 
 BOOST_AUTO_TEST_SUITE(ReferenceTestSuite)
 
+
 BOOST_AUTO_TEST_CASE(StdSharedPointerTestBefore)
 {
+#if !defined(BOOST_NO_CXX11_SMART_PTR)
     bool objAlive = false;
 
     std::shared_ptr<LivenessMarker> shared = std::make_shared<LivenessMarker>(objAlive);
@@ -167,10 +169,12 @@ BOOST_AUTO_TEST_CASE(StdSharedPointerTestBefore)
     }
 
     BOOST_CHECK(!objAlive);
+#endif
 }
 
 BOOST_AUTO_TEST_CASE(StdSharedPointerTestAfter)
 {
+#if !defined(BOOST_NO_CXX11_SMART_PTR)
     bool objAlive = false;
 
     std::shared_ptr<LivenessMarker> shared = std::make_shared<LivenessMarker>(objAlive);
@@ -188,6 +192,7 @@ BOOST_AUTO_TEST_CASE(StdSharedPointerTestAfter)
     shared.reset();
 
     BOOST_CHECK(!objAlive);
+#endif
 }
 
 BOOST_AUTO_TEST_CASE(StdAutoPointerTest)
@@ -209,6 +214,7 @@ BOOST_AUTO_TEST_CASE(StdAutoPointerTest)
 
 BOOST_AUTO_TEST_CASE(StdUniquePointerTest)
 {
+#if !defined(BOOST_NO_CXX11_SMART_PTR)
     bool objAlive = false;
 
     std::unique_ptr<LivenessMarker> unique(new LivenessMarker(objAlive));
@@ -222,13 +228,14 @@ BOOST_AUTO_TEST_CASE(StdUniquePointerTest)
     }
 
     BOOST_CHECK(!objAlive);
+#endif
 }
 
 BOOST_AUTO_TEST_CASE(BoostSharedPointerTestBefore)
 {
     bool objAlive = false;
 
-    boost::shared_ptr<LivenessMarker> shared = boost::make_shared<LivenessMarker>(objAlive);
+    boost::shared_ptr<LivenessMarker> shared = boost::make_shared<LivenessMarker>(boost::ref(objAlive));
 
     BOOST_CHECK(objAlive);
 
@@ -249,7 +256,7 @@ BOOST_AUTO_TEST_CASE(BoostSharedPointerTestAfter)
 {
     bool objAlive = false;
 
-    boost::shared_ptr<LivenessMarker> shared = boost::make_shared<LivenessMarker>(objAlive);
+    boost::shared_ptr<LivenessMarker> shared = boost::make_shared<LivenessMarker>(boost::ref(objAlive));
 
     BOOST_CHECK(objAlive);
 
@@ -266,8 +273,10 @@ BOOST_AUTO_TEST_CASE(BoostSharedPointerTestAfter)
     BOOST_CHECK(!objAlive);
 }
 
+
 BOOST_AUTO_TEST_CASE(PassingToFunction)
 {
+#if !defined(BOOST_NO_CXX11_SMART_PTR)
     bool objAlive = false;
 
     std::shared_ptr<LivenessMarker> stdShared = std::make_shared<LivenessMarker>(objAlive);
@@ -281,6 +290,7 @@ BOOST_AUTO_TEST_CASE(PassingToFunction)
     TestFunction(MakeReferenceFromSmartPointer(stdAuto));
 
     TestFunction(MakeReferenceFromSmartPointer(boostShared));
+#endif
 }
 
 BOOST_AUTO_TEST_CASE(CopyTest)

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/core/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/Makefile.am b/modules/platforms/cpp/core/Makefile.am
index 758e68d..56e0c0a 100644
--- a/modules/platforms/cpp/core/Makefile.am
+++ b/modules/platforms/cpp/core/Makefile.am
@@ -37,7 +37,7 @@ AM_CPPFLAGS = \
 
 AM_CXXFLAGS = \
     -Wall \
-    -std=c++0x
+    -std=c++03
 
 libignite_la_LIBADD = \
     @top_srcdir@/jni/libignite-jni.la \

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h b/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h
index 4bd9c46..781ce2e 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h
@@ -82,7 +82,7 @@ namespace ignite
                      * @param lsnr Event listener. Invoked on the node where
                      *     continuous query execution has been started.
                      */
-                    ContinuousQuery(Reference<event::CacheEntryEventListener<K, V>> lsnr) :
+                    ContinuousQuery(Reference<event::CacheEntryEventListener<K, V> > lsnr) :
                         impl(new impl::cache::query::continuous::ContinuousQueryImpl<K, V>(lsnr))
                     {
                         // No-op.
@@ -95,7 +95,7 @@ namespace ignite
                      *     continuous query execution has been started.
                      * @param loc Whether query should be executed locally.
                      */
-                    ContinuousQuery(Reference<event::CacheEntryEventListener<K, V>> lsnr, bool loc) :
+                    ContinuousQuery(Reference<event::CacheEntryEventListener<K, V> > lsnr, bool loc) :
                         impl(new impl::cache::query::continuous::ContinuousQueryImpl<K, V>(lsnr, loc))
                     {
                         // No-op.
@@ -202,7 +202,7 @@ namespace ignite
                      *     node where continuous query execution has been
                      *     started.
                      */
-                    void SetListener(Reference<event::CacheEntryEventListener<K, V>> lsnr)
+                    void SetListener(Reference<event::CacheEntryEventListener<K, V> > lsnr)
                     {
                         impl.Get()->SetListener(lsnr);
                     }
@@ -229,7 +229,7 @@ namespace ignite
 
                 private:
                     /** Implementation. */
-                    common::concurrent::SharedPointer<impl::cache::query::continuous::ContinuousQueryImpl<K, V>> impl;
+                    common::concurrent::SharedPointer<impl::cache::query::continuous::ContinuousQueryImpl<K, V> > impl;
                 };
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/core/include/ignite/cache/query/query_cursor.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/query_cursor.h b/modules/platforms/cpp/core/include/ignite/cache/query/query_cursor.h
index 61c6813..9057a03 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/query/query_cursor.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/query/query_cursor.h
@@ -190,7 +190,7 @@ namespace ignite
                  *
                  * @throw IgniteError class instance in case of failure.
                  */
-                void GetAll(std::vector<CacheEntry<K, V>>& res)
+                void GetAll(std::vector<CacheEntry<K, V> >& res)
                 {
                     IgniteError err;
 
@@ -208,7 +208,7 @@ namespace ignite
                  * @param Vector where query entries will be stored.
                  * @param err Used to set operation result.
                  */
-                void GetAll(std::vector<CacheEntry<K, V>>& res, IgniteError& err)
+                void GetAll(std::vector<CacheEntry<K, V> >& res, IgniteError& err)
                 {
                     impl::cache::query::QueryCursorImpl* impl0 = impl.Get();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_impl.h b/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_impl.h
index 50ced12..2a24e5f 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_impl.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/cache/query/continuous/continuous_query_impl.h
@@ -255,7 +255,7 @@ namespace ignite
                          * @param lsnr Event listener. Invoked on the node where
                          *     continuous query execution has been started.
                          */
-                        ContinuousQueryImpl(Reference<ignite::cache::event::CacheEntryEventListener<K, V>>& lsnr) :
+                        ContinuousQueryImpl(Reference<ignite::cache::event::CacheEntryEventListener<K, V> >& lsnr) :
                             ContinuousQueryImplBase(false),
                             lsnr(lsnr)
                         {
@@ -269,7 +269,7 @@ namespace ignite
                          *     continuous query execution has been started.
                          * @param loc Whether query should be executed locally.
                          */
-                        ContinuousQueryImpl(Reference<ignite::cache::event::CacheEntryEventListener<K, V>>& lsnr, bool loc) :
+                        ContinuousQueryImpl(Reference<ignite::cache::event::CacheEntryEventListener<K, V> >& lsnr, bool loc) :
                             ContinuousQueryImplBase(loc),
                             lsnr(lsnr)
                         {
@@ -283,7 +283,7 @@ namespace ignite
                          *     node where continuous query execution has been
                          *     started.
                          */
-                        void SetListener(Reference<ignite::cache::event::CacheEntryEventListener<K, V>>& val)
+                        void SetListener(Reference<ignite::cache::event::CacheEntryEventListener<K, V> >& val)
                         {
                             lsnr = val;
                         }
@@ -340,7 +340,7 @@ namespace ignite
 
                     private:
                         /** Cache entry event listener. */
-                        Reference<ignite::cache::event::CacheEntryEventListener<K, V>> lsnr;
+                        Reference<ignite::cache::event::CacheEntryEventListener<K, V> > lsnr;
                     };
                 }
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/core/include/ignite/impl/handle_registry.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/handle_registry.h b/modules/platforms/cpp/core/include/ignite/impl/handle_registry.h
index 3c4d123..ecaf3ee 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/handle_registry.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/handle_registry.h
@@ -72,7 +72,7 @@ namespace ignite
             void Clear();
         private:
             /** Map with data. */
-            std::map<int64_t, common::concurrent::SharedPointer<void>> map;
+            std::map<int64_t, common::concurrent::SharedPointer<void> > map;
 
             /** Mutex. */
             common::concurrent::CriticalSection mux;

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/core/include/ignite/impl/operations.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/impl/operations.h b/modules/platforms/cpp/core/include/ignite/impl/operations.h
index a8fef93..177529d 100644
--- a/modules/platforms/cpp/core/include/ignite/impl/operations.h
+++ b/modules/platforms/cpp/core/include/ignite/impl/operations.h
@@ -531,7 +531,7 @@ namespace ignite
             /**
              * Constructor.
              */
-            OutQueryGetAllOperation(std::vector<ignite::cache::CacheEntry<K, V>>* res) : res(res)
+            OutQueryGetAllOperation(std::vector<ignite::cache::CacheEntry<K, V> >* res) : res(res)
             {
                 // No-op.
             }
@@ -556,7 +556,7 @@ namespace ignite
 
         private:
             /** Entries. */
-            std::vector<ignite::cache::CacheEntry<K, V>>* res;
+            std::vector<ignite::cache::CacheEntry<K, V> >* res;
             
             IGNITE_NO_COPY_ASSIGNMENT(OutQueryGetAllOperation)
         };

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp b/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp
index ad69d45..5d09e46 100644
--- a/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp
+++ b/modules/platforms/cpp/core/src/impl/cache/cache_impl.cpp
@@ -325,11 +325,12 @@ namespace ignite
                 return QueryContinuous(qry, initialQry, OP_QRY_SCAN, OP_QRY_CONTINUOUS, err);
             }
 
+            struct DummyQry { void Write(BinaryRawWriter&) const { }};
+
             ContinuousQueryHandleImpl* CacheImpl::QueryContinuous(const SharedPointer<ContinuousQueryImplBase> qry,
                 IgniteError& err)
             {
-                struct { void Write(BinaryRawWriter&) const { }} dummy;
-
+                DummyQry dummy;
                 return QueryContinuous(qry, dummy, -1, OP_QRY_CONTINUOUS, err);
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/core/src/impl/handle_registry.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/impl/handle_registry.cpp b/modules/platforms/cpp/core/src/impl/handle_registry.cpp
index 069e996..a279762 100644
--- a/modules/platforms/cpp/core/src/impl/handle_registry.cpp
+++ b/modules/platforms/cpp/core/src/impl/handle_registry.cpp
@@ -37,7 +37,7 @@ namespace ignite
 
         SharedPointer<void> HandleRegistrySegment::Get(int64_t hnd)
         {
-            typedef std::map<int64_t, SharedPointer<void>> Map;
+            typedef std::map<int64_t, SharedPointer<void> > Map;
 
             CsLockGuard guard(mux);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/examples/odbc-example/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/odbc-example/Makefile.am b/modules/platforms/cpp/examples/odbc-example/Makefile.am
index 83cc63e..f3c9a50 100644
--- a/modules/platforms/cpp/examples/odbc-example/Makefile.am
+++ b/modules/platforms/cpp/examples/odbc-example/Makefile.am
@@ -36,7 +36,7 @@ AM_CPPFLAGS = \
 
 AM_CXXFLAGS = \
     -Wall \
-    -std=c++0x
+    -std=c++03
 
 ignite_odbcexample_LDADD = \
     @top_srcdir@/../core/libignite.la \

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/examples/putget-example/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/putget-example/Makefile.am b/modules/platforms/cpp/examples/putget-example/Makefile.am
index cf39002..88edc81 100644
--- a/modules/platforms/cpp/examples/putget-example/Makefile.am
+++ b/modules/platforms/cpp/examples/putget-example/Makefile.am
@@ -36,7 +36,7 @@ AM_CPPFLAGS = \
 
 AM_CXXFLAGS = \
     -Wall \
-    -std=c++0x
+    -std=c++03
 
 ignite_putgetexample_LDADD = \
     @top_srcdir@/../core/libignite.la \

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/examples/query-example/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/query-example/Makefile.am b/modules/platforms/cpp/examples/query-example/Makefile.am
index 01231ec..1d00bcf 100644
--- a/modules/platforms/cpp/examples/query-example/Makefile.am
+++ b/modules/platforms/cpp/examples/query-example/Makefile.am
@@ -36,7 +36,7 @@ AM_CPPFLAGS = \
 
 AM_CXXFLAGS = \
     -Wall \
-    -std=c++0x
+    -std=c++03
 
 ignite_queryexample_LDADD = \
     @top_srcdir@/../core/libignite.la \

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/ignite/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/ignite/Makefile.am b/modules/platforms/cpp/ignite/Makefile.am
index 2dbc4d6..9b65f05 100644
--- a/modules/platforms/cpp/ignite/Makefile.am
+++ b/modules/platforms/cpp/ignite/Makefile.am
@@ -36,7 +36,7 @@ AM_CPPFLAGS = \
 
 AM_CXXFLAGS = \
     -Wall \
-    -std=c++0x
+    -std=c++03
 
 ignite_LDADD = \
     @top_srcdir@/core/libignite.la \

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/jni/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/Makefile.am b/modules/platforms/cpp/jni/Makefile.am
index 2cb4b90..56eaa6c 100644
--- a/modules/platforms/cpp/jni/Makefile.am
+++ b/modules/platforms/cpp/jni/Makefile.am
@@ -35,7 +35,7 @@ AM_CPPFLAGS = \
 
 AM_CXXFLAGS = \
     -Wall \
-    -std=c++0x
+    -std=c++03
 
 libignite_jni_la_LIBADD = \
     -L$(JAVA_HOME)/jre/lib/amd64/server \

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/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 e6f2f88..2c8c76a 100644
--- a/modules/platforms/cpp/jni/src/java.cpp
+++ b/modules/platforms/cpp/jni/src/java.cpp
@@ -1552,7 +1552,7 @@ namespace ignite
                 if (consoleWriteHandlers.size() > 0) {
                     ConsoleWriteHandler consoleWrite = consoleWriteHandlers.at(0);
 
-                    const char* strChars = env->GetStringUTFChars(str, nullptr);
+                    const char* strChars = env->GetStringUTFChars(str, 0);
                     const int strCharsLen = env->GetStringUTFLength(str);
 
                     consoleWrite(strChars, strCharsLen, isErr);

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/odbc-test/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am
index e10ef7a..c2f3fa2 100644
--- a/modules/platforms/cpp/odbc-test/Makefile.am
+++ b/modules/platforms/cpp/odbc-test/Makefile.am
@@ -40,7 +40,7 @@ AM_CPPFLAGS = \
 
 AM_CXXFLAGS = \
     -Wall \
-    -std=c++0x
+    -std=c++03
 
 ignite_odbc_tests_LDADD = \
     @top_srcdir@/core/libignite.la \

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/odbc-test/config/queries-default.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/config/queries-default.xml b/modules/platforms/cpp/odbc-test/config/queries-default.xml
index 6614e93..6dbc4f7 100644
--- a/modules/platforms/cpp/odbc-test/config/queries-default.xml
+++ b/modules/platforms/cpp/odbc-test/config/queries-default.xml
@@ -59,6 +59,7 @@
                                         <entry key="guidField" value="java.util.UUID"/>
                                         <entry key="dateField" value="java.util.Date"/>
                                         <entry key="timestampField" value="java.sql.Timestamp"/>
+                                        <entry key="i8ArrayField" value="[B"/>
                                     </map>
                                 </property>
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/odbc-test/include/sql_test_suite_fixture.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/include/sql_test_suite_fixture.h b/modules/platforms/cpp/odbc-test/include/sql_test_suite_fixture.h
index 6d26818..55353e5 100644
--- a/modules/platforms/cpp/odbc-test/include/sql_test_suite_fixture.h
+++ b/modules/platforms/cpp/odbc-test/include/sql_test_suite_fixture.h
@@ -192,6 +192,9 @@ namespace ignite
 
     template<>
     void SqlTestSuiteFixture::CheckSingleResult<Timestamp>(const char* request);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<std::vector<int8_t> >(const char* request, const std::vector<int8_t>& expected);
 }
 
 #endif //_IGNITE_ODBC_TEST_SQL_TEST_SUIT_FIXTURE

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/odbc-test/include/test_type.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/include/test_type.h b/modules/platforms/cpp/odbc-test/include/test_type.h
index 0e08251..daeff83 100644
--- a/modules/platforms/cpp/odbc-test/include/test_type.h
+++ b/modules/platforms/cpp/odbc-test/include/test_type.h
@@ -74,6 +74,7 @@ namespace ignite
         Guid guidField;
         Date dateField;
         Timestamp timestampField;
+        std::vector<int8_t> i8ArrayField;
     };
 }
 
@@ -107,6 +108,14 @@ namespace ignite
                     writer.WriteGuid("guidField", obj.guidField);
                     writer.WriteDate("dateField", obj.dateField);
                     writer.WriteTimestamp("timestampField", obj.timestampField);
+                    if (obj.i8ArrayField.empty())
+                    {
+                        writer.WriteNull("i8ArrayField");
+                    }
+                    else
+                    {
+                        writer.WriteInt8Array("i8ArrayField", &obj.i8ArrayField[0], static_cast<int32_t>(obj.i8ArrayField.size()));
+                    }
                 }
                 else
                 {
@@ -121,6 +130,7 @@ namespace ignite
                     writer.WriteNull("guidField");
                     writer.WriteNull("dateField");
                     writer.WriteNull("timestampField");
+                    writer.WriteNull("i8ArrayField");
                 }
             }
 
@@ -138,9 +148,17 @@ namespace ignite
                 Date dateField = reader.ReadDate("dateField");
                 Timestamp timestampField = reader.ReadTimestamp("timestampField");
 
-                return TestType(i8Field, i16Field, i32Field, i64Field, strField,
+                TestType result(i8Field, i16Field, i32Field, i64Field, strField,
                     floatField, doubleField, boolField, guidField, dateField,
                     timestampField);
+
+                int32_t len = reader.ReadInt8Array("i8ArrayField", 0, 0);
+                if (len > 0)
+                {
+                    result.i8ArrayField.resize(len);
+                    reader.ReadInt8Array("i8ArrayField", &result.i8ArrayField[0], len);
+                }
+                return result;
             }
 
         IGNITE_BINARY_TYPE_END

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/odbc-test/src/column_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/column_test.cpp b/modules/platforms/cpp/odbc-test/src/column_test.cpp
index 6cbea8b..66d0214 100644
--- a/modules/platforms/cpp/odbc-test/src/column_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/column_test.cpp
@@ -287,4 +287,193 @@ BOOST_AUTO_TEST_CASE(TestColumnMultiString)
     BOOST_REQUIRE(column2.GetUnreadDataLength() == 0);
 }
 
+BOOST_AUTO_TEST_CASE(TestColumnByteArray)
+{
+    ignite::impl::interop::InteropUnpooledMemory mem(4096);
+    ignite::impl::interop::InteropOutputStream outStream(&mem);
+    ignite::impl::binary::BinaryWriterImpl writer(&outStream, 0);
+
+    const int8_t bytes[] = { 'A','B','C','D','E','F','G','H','I','J' };
+    std::vector<int8_t> data(bytes, bytes + sizeof(bytes) / sizeof(bytes[0]));
+    writer.WriteInt8Array(&data[0], static_cast<int32_t>(data.size()));
+
+    outStream.Synchronize();
+
+    ignite::impl::interop::InteropInputStream inStream(&mem);
+    ignite::impl::binary::BinaryReaderImpl reader(&inStream);
+
+    Column column(reader);
+
+    BOOST_REQUIRE(column.IsValid());
+
+    BOOST_REQUIRE(column.GetSize() == data.size());
+
+    BOOST_REQUIRE(column.GetUnreadDataLength() == data.size());
+
+    std::vector<int8_t> buf(data.size());
+    SqlLen reslen = 0;
+    int* offset = 0;
+
+    ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_BINARY, &buf[0], buf.size(), &reslen, &offset);
+
+    BOOST_REQUIRE(column.ReadToBuffer(reader, appBuf) == SQL_RESULT_SUCCESS);
+
+    BOOST_REQUIRE(column.IsValid());
+
+    BOOST_REQUIRE(column.GetSize() == data.size());
+
+    BOOST_REQUIRE(column.GetUnreadDataLength() == 0);
+
+    BOOST_REQUIRE_EQUAL_COLLECTIONS(buf.begin(), buf.end(), data.begin(), data.end());
+
+    BOOST_REQUIRE(column.ReadToBuffer(reader, appBuf) == SQL_RESULT_NO_DATA);
+
+    BOOST_REQUIRE(column.IsValid());
+
+    BOOST_REQUIRE(column.GetSize() == data.size());
+
+    BOOST_REQUIRE(column.GetUnreadDataLength() == 0);
+}
+
+BOOST_AUTO_TEST_CASE(TestColumnByteArrayHalfBuffer)
+{
+    ignite::impl::interop::InteropUnpooledMemory mem(4096);
+    ignite::impl::interop::InteropOutputStream outStream(&mem);
+    ignite::impl::binary::BinaryWriterImpl writer(&outStream, 0);
+
+    const int8_t bytes[] = { 'A','B','C','D','E','F','G','H','I','J' };
+    std::vector<int8_t> data(bytes, bytes + sizeof(bytes)/sizeof(bytes[0]));
+    BOOST_REQUIRE(0 == data.size() % 2);
+
+    writer.WriteInt8Array(&data[0], static_cast<int32_t>(data.size()));
+
+    outStream.Synchronize();
+
+    ignite::impl::interop::InteropInputStream inStream(&mem);
+    ignite::impl::binary::BinaryReaderImpl reader(&inStream);
+
+    Column column(reader);
+
+    BOOST_REQUIRE(column.IsValid());
+
+    BOOST_REQUIRE(column.GetSize() == data.size());
+
+    BOOST_REQUIRE(column.GetUnreadDataLength() == data.size());
+
+    std::vector<int8_t> buf(data.size()/2);
+    SqlLen reslen = 0;
+    int* offset = 0;
+
+    ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_BINARY, &buf[0], buf.size(), &reslen, &offset);
+
+    BOOST_REQUIRE(column.ReadToBuffer(reader, appBuf) == SQL_RESULT_SUCCESS);
+
+    BOOST_REQUIRE(column.IsValid());
+
+    BOOST_REQUIRE(column.GetSize() == data.size());
+
+    BOOST_REQUIRE(column.GetUnreadDataLength() == data.size() - buf.size());
+
+    std::vector<int8_t> result;
+    result.reserve(data.size());
+    std::copy(buf.begin(), buf.end(), std::back_inserter(result));
+
+    BOOST_REQUIRE(column.ReadToBuffer(reader, appBuf) == SQL_RESULT_SUCCESS);
+
+    BOOST_REQUIRE(column.IsValid());
+
+    BOOST_REQUIRE(column.GetSize() == data.size());
+
+    BOOST_REQUIRE(column.GetUnreadDataLength() == 0);
+    
+    std::copy(buf.begin(), buf.end(), std::back_inserter(result));
+
+    BOOST_REQUIRE(column.ReadToBuffer(reader, appBuf) == SQL_RESULT_NO_DATA);
+
+    BOOST_REQUIRE(column.IsValid());
+
+    BOOST_REQUIRE(column.GetSize() == data.size());
+
+    BOOST_REQUIRE(column.GetUnreadDataLength() == 0);
+
+    BOOST_REQUIRE_EQUAL_COLLECTIONS(result.begin(), result.end(), data.begin(), data.end());
+}
+
+BOOST_AUTO_TEST_CASE(TestColumnByteArrayTwoColumns)
+{
+    ignite::impl::interop::InteropUnpooledMemory mem(4096);
+    ignite::impl::interop::InteropOutputStream outStream(&mem);
+    ignite::impl::binary::BinaryWriterImpl writer(&outStream, 0);
+
+    const int8_t bytes1[] = { 'A','B','C','D','E','F','G','H','I','J' };
+    const int8_t bytes2[] = { 'a','b','c','d','e','f','g','h','i','j' };    
+    std::vector<int8_t> data1(bytes1, bytes1 + sizeof(bytes1)/sizeof(bytes1[0]));
+    std::vector<int8_t> data2(bytes2, bytes2 + sizeof(bytes2)/sizeof(bytes2[0]));
+    writer.WriteInt8Array(&data1[0], static_cast<int32_t>(data1.size()));
+    writer.WriteInt8Array(&data2[0], static_cast<int32_t>(data2.size()));
+
+    outStream.Synchronize();
+
+    ignite::impl::interop::InteropInputStream inStream(&mem);
+    ignite::impl::binary::BinaryReaderImpl reader(&inStream);
+
+    Column column1(reader);
+    inStream.Position(column1.GetEndPosition());
+    Column column2(reader);
+
+    BOOST_REQUIRE(column1.IsValid());
+
+    BOOST_REQUIRE(column1.GetSize() == data1.size());
+
+    BOOST_REQUIRE(column1.GetUnreadDataLength() == data1.size());
+
+    BOOST_REQUIRE(column2.IsValid());
+
+    BOOST_REQUIRE(column2.GetSize() == data2.size());
+
+    BOOST_REQUIRE(column2.GetUnreadDataLength() == data2.size());
+
+    int8_t buf[1024] = {};
+    SqlLen reslen = 0;
+    int* offset = 0;
+
+    ApplicationDataBuffer appBuf(type_traits::IGNITE_ODBC_C_TYPE_BINARY, &buf, sizeof(buf), &reslen, &offset);
+
+    BOOST_REQUIRE(column1.ReadToBuffer(reader, appBuf) == SQL_RESULT_SUCCESS);
+
+    BOOST_REQUIRE(column1.IsValid());
+
+    BOOST_REQUIRE(column1.GetSize() == data1.size());
+
+    BOOST_REQUIRE(column1.GetUnreadDataLength() == 0);
+
+    BOOST_REQUIRE_EQUAL_COLLECTIONS(buf, buf + reslen, data1.begin(), data1.end());
+
+    BOOST_REQUIRE(column1.ReadToBuffer(reader, appBuf) == SQL_RESULT_NO_DATA);
+
+    BOOST_REQUIRE(column1.IsValid());
+
+    BOOST_REQUIRE(column1.GetSize() == data1.size());
+
+    BOOST_REQUIRE(column1.GetUnreadDataLength() == 0);
+
+    BOOST_REQUIRE(column2.ReadToBuffer(reader, appBuf) == SQL_RESULT_SUCCESS);
+
+    BOOST_REQUIRE(column2.IsValid());
+
+    BOOST_REQUIRE(column2.GetSize() == data1.size());
+
+    BOOST_REQUIRE(column2.GetUnreadDataLength() == 0);
+
+    BOOST_REQUIRE_EQUAL_COLLECTIONS(buf, buf + reslen, data2.begin(), data2.end());
+
+    BOOST_REQUIRE(column2.ReadToBuffer(reader, appBuf) == SQL_RESULT_NO_DATA);
+
+    BOOST_REQUIRE(column2.IsValid());
+
+    BOOST_REQUIRE(column2.GetSize() == data1.size());
+
+    BOOST_REQUIRE(column2.GetUnreadDataLength() == 0);
+}
+
 BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp b/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp
index 29d1d18..b118ff8 100644
--- a/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp
+++ b/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp
@@ -285,4 +285,21 @@ namespace ignite
 
         CheckSingleResult0(request, SQL_C_TIMESTAMP, &res, 0, 0);
     }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<std::vector<int8_t> >(const char* request, const std::vector<int8_t>& expected)
+    {
+        SQLCHAR res[ODBC_BUFFER_SIZE] = { 0 };
+        SQLLEN resLen = 0;
+
+        CheckSingleResult0(request, SQL_C_BINARY, res, ODBC_BUFFER_SIZE, &resLen);
+
+        BOOST_REQUIRE_EQUAL(resLen, expected.size());
+
+        if (resLen > 0)
+        {
+            std::vector<int8_t> actual(res, res + resLen);
+            BOOST_REQUIRE_EQUAL_COLLECTIONS(expected.begin(), expected.end(), actual.begin(), actual.end());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/odbc-test/src/sql_types_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_types_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_types_test.cpp
index ccbb917..33797b0 100644
--- a/modules/platforms/cpp/odbc-test/src/sql_types_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/sql_types_test.cpp
@@ -22,6 +22,7 @@
 #include <boost/test/unit_test.hpp>
 
 #include "sql_test_suite_fixture.h"
+#include "test_utils.h"
 
 using namespace ignite;
 
@@ -56,5 +57,135 @@ BOOST_AUTO_TEST_CASE(TestGuidEqualsToColumn)
         "SELECT i32Field FROM TestType WHERE guidField = {guid '04cc382a-0b82-f520-08d0-07a0620c0004'}", in2.i32Field);
 }
 
+BOOST_AUTO_TEST_CASE(TestByteArraySelect)
+{
+    TestType in;
+    const int8_t data[] = { 'A','B','C','D','E','F','G','H','I','J' };
+    in.i8ArrayField.assign(data, data + sizeof(data)/sizeof(data[0]));
+    testCache.Put(1, in);
+
+    TestType out = testCache.Get(1);
+
+    BOOST_REQUIRE(in.i8ArrayField.size() == out.i8ArrayField.size());
+
+    BOOST_REQUIRE_EQUAL_COLLECTIONS(in.i8ArrayField.begin(), in.i8ArrayField.end(), out.i8ArrayField.begin(), out.i8ArrayField.end());
+
+    CheckSingleResult<std::vector<int8_t> >("SELECT i8ArrayField FROM TestType", in.i8ArrayField);
+}
+
+BOOST_AUTO_TEST_CASE(TestByteArrayParam)
+{
+    SQLRETURN ret;
+    
+    TestType in;
+    in.i8Field = 101;
+
+    const int8_t data[] = { 'A','B','C','D','E','F','G','H','I','J' };
+    in.i8ArrayField.assign(data, data + sizeof(data) / sizeof(data[0]));
+
+    testCache.Put(1, in);   
+
+    SQLLEN colLen = 0;
+    SQLCHAR colData = 0;
+
+    ret = SQLBindCol(stmt, 1, SQL_C_TINYINT, &colData, sizeof(colData), &colLen);
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    SQLCHAR request[] = "SELECT i8Field FROM TestType WHERE i8ArrayField = ?";
+
+    ret = SQLPrepare(stmt, request, SQL_NTS);
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    std::vector<int8_t> paramData(in.i8ArrayField);
+    SQLLEN paramLen = paramData.size();
+    ret = SQLBindParameter(stmt, 1, SQL_PARAM_INPUT, SQL_C_BINARY, SQL_VARBINARY, paramData.size(), 0, &paramData[0], paramData.size(), &paramLen);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    ret = SQLExecute(stmt);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    ret = SQLFetch(stmt);
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    BOOST_REQUIRE_EQUAL(colData, in.i8Field);
+    BOOST_REQUIRE_EQUAL(colLen, sizeof(colData));
+
+    ret = SQLFetch(stmt);
+    BOOST_REQUIRE(ret == SQL_NO_DATA);
+}
+
+BOOST_AUTO_TEST_CASE(TestByteArrayParamInsert)
+{
+    SQLRETURN ret;
+
+    const int8_t data[] = { 'A','B','C','D','E','F','G','H','I','J' };
+    std::vector<int8_t> paramData(data, data + sizeof(data) / sizeof(data[0]));
+    SQLCHAR request[] = "INSERT INTO TestType(_key, i8ArrayField) VALUES(?, ?)";;
+
+    ret = SQLPrepare(stmt, request, SQL_NTS);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    int64_t key = 1;
+    ret = SQLBindParameter(stmt, 1, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_BIGINT, 0, 0, &key, 0, 0);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+    
+    SQLLEN paramLen = paramData.size();
+
+    ret = SQLBindParameter(stmt, 2, SQL_PARAM_INPUT, SQL_C_BINARY, SQL_VARBINARY, paramData.size(), 0, &paramData[0], paramData.size(), &paramLen);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    ret = SQLExecute(stmt);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    TestType out = testCache.Get(key);
+    BOOST_REQUIRE_EQUAL_COLLECTIONS(out.i8ArrayField.begin(), out.i8ArrayField.end(), paramData.begin(), paramData.end());
+}
+
+BOOST_AUTO_TEST_CASE(TestByteParamInsert)
+{
+    SQLRETURN ret;
+
+    SQLCHAR request[] = "INSERT INTO TestType(_key, i8Field) VALUES(?, ?)";;
+
+    ret = SQLPrepare(stmt, request, SQL_NTS);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    int64_t key = 1;
+    ret = SQLBindParameter(stmt, 1, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_BIGINT, 0, 0, &key, 0, 0);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    int8_t data = 2;
+    ret = SQLBindParameter(stmt, 2, SQL_PARAM_INPUT, SQL_C_TINYINT, SQL_TINYINT, 0, 0, &data, 0, 0);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    ret = SQLExecute(stmt);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    TestType out = testCache.Get(key);
+    BOOST_REQUIRE_EQUAL(out.i8Field, data);
+}
 
 BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/odbc/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/Makefile.am b/modules/platforms/cpp/odbc/Makefile.am
index b0cc5f8..1eb42d1 100644
--- a/modules/platforms/cpp/odbc/Makefile.am
+++ b/modules/platforms/cpp/odbc/Makefile.am
@@ -33,7 +33,7 @@ AM_CPPFLAGS = \
 
 AM_CXXFLAGS = \
     -Wall \
-    -std=c++0x
+    -std=c++03
 
 libignite_odbc_la_LIBADD = \
     @top_srcdir@/binary/libignite-binary.la \

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h b/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h
index 37daa50..590e97c 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h
@@ -46,7 +46,6 @@ void logInit(const char*);
         fprintf(log_file, "%s: " fmt, __FUNCTION__, ##__VA_ARGS__);   \
         fflush(log_file);                                           \
     } while (false)
-
 #else
 #   define LOG_MSG(...)
 #endif
@@ -181,6 +180,14 @@ namespace ignite
          * @return Standard string containing the same data.
          */
         std::string SqlStringToString(const unsigned char* sqlStr, int32_t sqlStrLen);
+
+        /**
+         * Convert binary data to hex dump form
+         * @param data  pointer to data
+         * @param count data length
+         * @return standard string containing the formated hex dump
+         */
+        std::string HexDump(const char* data, size_t count);
     }
 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/odbc/src/column.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/column.cpp b/modules/platforms/cpp/odbc/src/column.cpp
index 3e6bbca..0ec8078 100644
--- a/modules/platforms/cpp/odbc/src/column.cpp
+++ b/modules/platforms/cpp/odbc/src/column.cpp
@@ -40,7 +40,7 @@ namespace
 
                 break;
             }
-            
+
             case IGNITE_TYPE_OBJECT:
             {
                 int8_t protoVer = stream.ReadInt8();
@@ -88,6 +88,7 @@ namespace
             case IGNITE_TYPE_DOUBLE:
             case IGNITE_TYPE_BOOL:
             case IGNITE_HDR_NULL:
+            case IGNITE_TYPE_ARRAY_BYTE:
             {
                 // No-op.
                 break;
@@ -290,6 +291,17 @@ namespace ignite
                     break;
                 }
 
+                case IGNITE_TYPE_ARRAY_BYTE:
+                {
+                    sizeTmp = reader.ReadInt32();
+                    assert(sizeTmp >= 0);
+
+                    startPosTmp = stream->Position();
+                    stream->Position(stream->Position() + sizeTmp);
+
+                    break;
+                }
+
                 default:
                 {
                     // This is a fail case.
@@ -473,6 +485,20 @@ namespace ignite
                     break;
                 }
 
+                case IGNITE_TYPE_ARRAY_BYTE:
+                {
+                    stream->Position(startPos + offset);
+                    int32_t maxRead = std::min(GetUnreadDataLength(), static_cast<int32_t>(dataBuf.GetSize()));
+                    std::vector<int8_t> data(maxRead);
+
+                    stream->ReadInt8Array(&data[0], static_cast<int32_t>(data.size()));
+
+                    int32_t written = dataBuf.PutBinaryData(data.data(), data.size());
+
+                    IncreaseOffset(written);
+                    break;
+                }
+
                 default:
                 {
                     // This is a fail case. Return false.

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/odbc/src/config/connection_info.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp
index 341ab7f..7c612bc 100644
--- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp
+++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp
@@ -412,7 +412,7 @@ namespace ignite
 
                 StringInfoMap::const_iterator itStr = strParams.find(type);
 
-                if (itStr != strParams.cend()) 
+                if (itStr != strParams.end()) 
                 {
                     unsigned short strlen = static_cast<short>(
                         utility::CopyStringToBuffer(itStr->second, 
@@ -426,7 +426,7 @@ namespace ignite
 
                 UintInfoMap::const_iterator itInt = intParams.find(type);
 
-                if (itInt != intParams.cend())
+                if (itInt != intParams.end())
                 {
                     unsigned int *res = reinterpret_cast<unsigned int*>(buf);
 
@@ -437,7 +437,7 @@ namespace ignite
 
                 UshortInfoMap::const_iterator itShort = shortParams.find(type);
 
-                if (itShort != shortParams.cend())
+                if (itShort != shortParams.end())
                 {
                     unsigned short *res = reinterpret_cast<unsigned short*>(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/odbc/src/connection.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/connection.cpp b/modules/platforms/cpp/odbc/src/connection.cpp
index 38f8cba..2afa42d 100644
--- a/modules/platforms/cpp/odbc/src/connection.cpp
+++ b/modules/platforms/cpp/odbc/src/connection.cpp
@@ -194,6 +194,8 @@ namespace ignite
 
             if (sent != len + sizeof(OdbcProtocolHeader))
                 IGNITE_ERROR_1(IgniteError::IGNITE_ERR_GENERIC, "Can not send message");
+
+            LOG_MSG("message sent: (%d bytes)%s\n", msg.GetSize(), utility::HexDump((char*)msg.GetData(), msg.GetSize()).c_str());
         }
 
         size_t Connection::SendAll(const int8_t* data, size_t len)

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/odbc/src/odbc.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp
index 0b18a11..dbec55f 100644
--- a/modules/platforms/cpp/odbc/src/odbc.cpp
+++ b/modules/platforms/cpp/odbc/src/odbc.cpp
@@ -512,9 +512,10 @@ namespace ignite
         int32_t res = statement->GetColumnNumber();
 
         if (columnNum)
+        {
             *columnNum = static_cast<SQLSMALLINT>(res);
-
-        LOG_MSG("columnNum: %d\n", *columnNum);
+            LOG_MSG("columnNum: %d\n", *columnNum);
+        }
 
         return statement->GetDiagnosticRecords().GetReturnCode();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/32d2629e/modules/platforms/cpp/odbc/src/utility.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/utility.cpp b/modules/platforms/cpp/odbc/src/utility.cpp
index 756ab36..c15c23c 100644
--- a/modules/platforms/cpp/odbc/src/utility.cpp
+++ b/modules/platforms/cpp/odbc/src/utility.cpp
@@ -154,6 +154,21 @@ namespace ignite
             else
                 res.clear();
         }
+
+        std::string HexDump(const char* data, size_t count)
+        {
+            std::stringstream  dump;
+            size_t cnt = 0;
+            for(const char* p = data, *e = data + count; p != e; ++p)
+            {
+                if (cnt++ % 16 == 0)
+                {
+                    dump << std::endl;
+                }
+                dump << std::hex << std::setfill('0') << std::setw(2) << (int)*p << " ";
+            }
+            return dump.str();
+        }
     }
 }
 


[17/50] [abbrv] ignite git commit: GridTopic.TOPIC_HADOOP_MSG: new topic should be added at the end for compatibility with previous releases.

Posted by yz...@apache.org.
GridTopic.TOPIC_HADOOP_MSG: new topic should be added at the end for compatibility with previous releases.


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 8372e69dd01972a01dbd59689475f2d45c2b6d94
Parents: ce46c10
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 22 10:30:05 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 22 10:30:05 2016 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/internal/GridTopic.java  | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8372e69d/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
index 24ddcd2..2962540 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
@@ -91,16 +91,16 @@ public enum GridTopic {
     TOPIC_HADOOP,
 
     /** */
-    TOPIC_HADOOP_MSG,
-
-    /** */
     TOPIC_QUERY,
 
     /** */
     TOPIC_TX,
 
     /** */
-    TOPIC_IO_TEST;
+    TOPIC_IO_TEST,
+
+    /** */
+    TOPIC_HADOOP_MSG;
 
     /** Enum values. */
     private static final GridTopic[] VALS = values();


[38/50] [abbrv] ignite git commit: Fixed broken links.

Posted by yz...@apache.org.
Fixed broken links.


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 6ca86704c36029ff29f11e1dba10588b341aa31a
Parents: fdf1f4b
Author: Vasiliy Sisko <vs...@gridgain.com>
Authored: Fri Dec 23 18:58:47 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Dec 23 18:58:47 2016 +0700

----------------------------------------------------------------------
 modules/core/src/main/java/org/apache/ignite/IgniteLogger.java | 6 +++---
 .../main/java/org/apache/ignite/logger/java/JavaLogger.java    | 4 ++--
 .../testframework/junits/logger/GridTestLog4jLogger.java       | 4 ++--
 .../main/java/org/apache/ignite/logger/log4j/Log4JLogger.java  | 4 ++--
 .../org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java | 2 +-
 5 files changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6ca86704/modules/core/src/main/java/org/apache/ignite/IgniteLogger.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteLogger.java b/modules/core/src/main/java/org/apache/ignite/IgniteLogger.java
index f3afa99..a52b49b 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteLogger.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteLogger.java
@@ -23,8 +23,8 @@ import org.jetbrains.annotations.Nullable;
 /**
  * This interface defines basic logging functionality used throughout the system. We had to
  * abstract it out so that we can use whatever logging is used by the hosting environment.
- * Currently, <a target=_new href="http://logging.apache.org/log4j/docs/">log4j</a>,
- * <a target=_new href="http://www.jboss.org/developers/guides/logging">JBoss</a>,
+ * Currently, <a target=_new href="http://logging.apache.org/log4j/1.2/">log4j</a>,
+ * <a target=_new href="http://docs.jboss.org/hibernate/orm/4.3/topical/html/logging/Logging">JBoss</a>,
  * <a target=_new href="http://jakarta.apache.org/commons/logging/">JCL</a> and
  * console logging are provided as supported implementations.
  * <p>
@@ -158,4 +158,4 @@ public interface IgniteLogger {
      * @return Name of the file being logged to if one is configured or {@code null} otherwise.
      */
     public String fileName();
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ca86704/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java
index d5ff5e3..6aa7d38 100644
--- a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java
+++ b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java
@@ -86,7 +86,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_QUIET;
  *      ...
  *      cfg.setGridLogger(log);
  * </pre>
- * Please take a look at <a target=_new href="http://java.sun.com/j2se/1.4.2/docs/api20/java/util/logging/Logger.html">Logger javadoc</a>
+ * Please take a look at <a target=_new href="http://docs.oracle.com/javase/7/docs/api/java/util/logging/Logger.html">Logger javadoc</a>
  * for additional information.
  * <p>
  * It's recommended to use Ignite logger injection instead of using/instantiating
@@ -406,4 +406,4 @@ public class JavaLogger implements IgniteLogger, LoggerNodeIdAware {
 
         return null;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ca86704/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java
index 74f5160..6a46c7d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java
@@ -50,7 +50,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_QUIET;
 
 /**
  * Log4j-based implementation for logging. This logger should be used
- * by loaders that have prefer <a target=_new href="http://logging.apache.org/log4j/docs/">log4j</a>-based logging.
+ * by loaders that have prefer <a target=_new href="http://logging.apache.org/log4j/1.2/">log4j</a>-based logging.
  * <p>
  * Here is a typical example of configuring log4j logger in Ignite configuration file:
  * <pre name="code" class="xml">
@@ -521,4 +521,4 @@ public class GridTestLog4jLogger implements IgniteLogger, LoggerNodeIdAware {
     @Override public String toString() {
         return S.toString(GridTestLog4jLogger.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ca86704/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java
----------------------------------------------------------------------
diff --git a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java
index eaae2d4..d5b0f02 100644
--- a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java
+++ b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java
@@ -50,7 +50,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_QUIET;
 
 /**
  * Log4j-based implementation for logging. This logger should be used
- * by loaders that have prefer <a target=_new href="http://logging.apache.org/log4j/docs/">log4j</a>-based logging.
+ * by loaders that have prefer <a target=_new href="http://logging.apache.org/log4j/1.2/">log4j</a>-based logging.
  * <p>
  * Here is a typical example of configuring log4j logger in Ignite configuration file:
  * <pre name="code" class="xml">
@@ -532,4 +532,4 @@ public class Log4JLogger implements IgniteLogger, LoggerNodeIdAware, Log4jFileAw
             }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ca86704/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java
----------------------------------------------------------------------
diff --git a/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java b/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java
index 22d42db..30940e4 100644
--- a/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java
+++ b/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java
@@ -111,7 +111,7 @@ import org.jetbrains.annotations.Nullable;
  * {@code META-INF/} entry may contain {@code ignite.xml} file which is a
  * task descriptor file. The purpose of task descriptor XML file is to specify
  * all tasks to be deployed. This file is a regular
- * <a href="http://www.springframework.org/documentation">Spring</a> XML
+ * <a href="https://spring.io/docs">Spring</a> XML
  * definition file.  {@code META-INF/} entry may also contain any other file
  * specified by JAR format.
  * </li>


[03/50] [abbrv] ignite git commit: Null argument check for cache.putAll.

Posted by yz...@apache.org.
Null argument check for cache.putAll.


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

Branch: refs/heads/ignite-comm-balance-master
Commit: c0f1a219e0e423cb65f25a3968318495d29ab86f
Parents: cf7b6a0
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 20 12:35:51 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 20 12:35:51 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/cache/GridCacheAdapter.java  | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c0f1a219/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 5707b49..50869be 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -2664,6 +2664,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public void putAll(@Nullable final Map<? extends K, ? extends V> m) throws IgniteCheckedException {
+        A.notNull(m, "map");
+
         if (F.isEmpty(m))
             return;
 


[12/50] [abbrv] ignite git commit: IGNITE-1443: Implemented ContinuousQuery for C++

Posted by yz...@apache.org.
IGNITE-1443: Implemented ContinuousQuery for C++

This closes #1343


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 598b464f81d2e74dc0df62011e08f3a76a674db6
Parents: 700529a
Author: Igor Sapego <is...@gridgain.com>
Authored: Tue Dec 20 19:38:41 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Dec 20 19:38:41 2016 +0300

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 .../include/ignite/binary/binary_raw_reader.h   |  21 +
 .../ignite/impl/binary/binary_reader_impl.h     |   9 +-
 .../src/impl/binary/binary_reader_impl.cpp      |  17 +
 .../platforms/cpp/common/include/Makefile.am    |   5 +-
 .../common/include/ignite/common/concurrent.h   |  90 ++-
 .../include/ignite/common/reference_impl.h      | 286 +++++++++
 .../cpp/common/include/ignite/reference.h       | 564 +++++++++++++++++
 .../cpp/common/project/vs/common.vcxproj        |   2 +
 .../common/project/vs/common.vcxproj.filters    |   6 +
 modules/platforms/cpp/core-test/Makefile.am     |   7 +-
 .../core-test/config/cache-query-continuous.xml |  87 +++
 .../cpp/core-test/project/vs/core-test.vcxproj  |  10 +-
 .../project/vs/core-test.vcxproj.filters        |   6 +
 .../cpp/core-test/src/cache_query_test.cpp      |  76 +--
 .../cpp/core-test/src/continuous_query_test.cpp | 611 +++++++++++++++++++
 .../cpp/core-test/src/handle_registry_test.cpp  |  18 +-
 .../cpp/core-test/src/reference_test.cpp        | 412 +++++++++++++
 modules/platforms/cpp/core/Makefile.am          |   1 +
 .../cpp/core/include/ignite/cache/cache.h       | 104 +++-
 .../cpp/core/include/ignite/cache/cache_entry.h |  40 +-
 .../ignite/cache/event/cache_entry_event.h      | 139 +++++
 .../cache/event/cache_entry_event_listener.h    |  71 +++
 .../cache/query/continuous/continuous_query.h   | 239 ++++++++
 .../query/continuous/continuous_query_handle.h  | 133 ++++
 .../core/include/ignite/impl/cache/cache_impl.h | 116 +++-
 .../continuous/continuous_query_handle_impl.h   | 101 +++
 .../query/continuous/continuous_query_impl.h    | 351 +++++++++++
 .../core/include/ignite/impl/handle_registry.h  |  62 +-
 .../include/ignite/impl/ignite_environment.h    |  34 +-
 modules/platforms/cpp/core/namespaces.dox       |  74 ++-
 .../platforms/cpp/core/project/vs/core.vcxproj  |   7 +
 .../cpp/core/project/vs/core.vcxproj.filters    |  30 +
 .../cpp/core/src/impl/cache/cache_impl.cpp      |  31 +
 .../continuous/continuous_query_handle_impl.cpp |  96 +++
 .../cpp/core/src/impl/handle_registry.cpp       | 102 ++--
 .../cpp/core/src/impl/ignite_environment.cpp    |  98 ++-
 modules/platforms/cpp/examples/Makefile.am      |   1 +
 modules/platforms/cpp/examples/configure.ac     |   1 +
 .../continuous-query-example/Makefile.am        |  58 ++
 .../config/continuous-query-example.xml         |  52 ++
 .../project/vs/continuous-query-example.vcxproj | 110 ++++
 .../vs/continuous-query-example.vcxproj.filters |  35 ++
 .../src/continuous_query_example.cpp            | 142 +++++
 .../examples/include/ignite/examples/person.h   |   2 +-
 .../cpp/examples/project/vs/ignite-examples.sln |   6 +
 .../platforms/cpp/jni/include/ignite/jni/java.h |   2 +-
 modules/platforms/cpp/jni/src/java.cpp          |   4 +-
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |   4 +-
 modules/platforms/cpp/project/vs/ignite.slnrel  |   3 +
 .../platforms/cpp/project/vs/ignite_x86.slnrel  |   3 +
 51 files changed, 4216 insertions(+), 264 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index b534eb7..db6128e 100644
--- a/.gitignore
+++ b/.gitignore
@@ -51,6 +51,7 @@ git-patch-prop-local.sh
 *.opensdf
 *.db
 *.opendb
+.vs
 ipch/
 [Oo]bj/
 [Bb]in

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h
index 72aab55..d15848b 100644
--- a/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h
+++ b/modules/platforms/cpp/binary/include/ignite/binary/binary_raw_reader.h
@@ -398,6 +398,27 @@ namespace ignite
             {
                 return impl->ReadObject<T>();
             }
+
+            /**
+             * Try read object.
+             * Reads value, stores it to res and returns true if the value is
+             * not null. Otherwise just returns false.
+             *
+             * @param res Read value is placed here if non-null.
+             * @return True if the non-null value has been read and false
+             *     otherwise.
+             */
+            template<typename T>
+            bool TryReadObject(T& res)
+            {
+                if (impl->SkipIfNull())
+                    return false;
+
+                res = impl->ReadObject<T>();
+
+                return true;
+            }
+
         private:
             /** Implementation delegate. */
             ignite::impl::binary::BinaryReaderImpl* impl;  

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h
index 8c4b464..cd32203 100644
--- a/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h
+++ b/modules/platforms/cpp/binary/include/ignite/impl/binary/binary_reader_impl.h
@@ -723,6 +723,13 @@ namespace ignite
                 bool HasNextElement(int32_t id) const;
 
                 /**
+                 * Skip next value if it is the null.
+                 *
+                 * @return True if the null value has been detected and skipped.
+                 */
+                bool SkipIfNull();
+
+                /**
                  * Read element.
                  *
                  * @param id Session ID.
@@ -763,7 +770,7 @@ namespace ignite
                     *key = ReadTopObject<K>();
                     *val = ReadTopObject<V>();
                 }
-                
+
                 /**
                  * Read object.
                  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp b/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp
index fb75ba5..c128df6 100644
--- a/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp
+++ b/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp
@@ -663,6 +663,23 @@ namespace ignite
                 return elemId == id && elemRead < elemCnt;
             }
 
+            bool BinaryReaderImpl::SkipIfNull()
+            {
+                CheckRawMode(true);
+                CheckSingleMode(true);
+
+                InteropStreamPositionGuard<InteropInputStream> positionGuard(*stream);
+
+                int8_t hdr = stream->ReadInt8();
+
+                if (hdr != IGNITE_HDR_NULL)
+                    return false;
+
+                positionGuard.Release();
+
+                return true;
+            }
+
             void BinaryReaderImpl::SetRawMode()
             {
                 CheckRawMode(false);

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/common/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/Makefile.am b/modules/platforms/cpp/common/include/Makefile.am
index 2e53608..a5073f6 100644
--- a/modules/platforms/cpp/common/include/Makefile.am
+++ b/modules/platforms/cpp/common/include/Makefile.am
@@ -23,14 +23,15 @@ nobase_include_HEADERS = \
 	ignite/common/concurrent.h \
 	ignite/common/decimal.h \
 	ignite/common/default_allocator.h \
+	ignite/common/reference_impl.h \
 	ignite/common/dynamic_size_array.h \
 	ignite/common/fixed_size_array.h \
 	ignite/common/utils.h \
 	ignite/date.h \
 	ignite/guid.h \
 	ignite/ignite_error.h \
-	ignite/timestamp.h
-	ignite/timestamp.h
+	ignite/timestamp.h \
+	ignite/reference.h
 
 uninstall-hook:
 	if [ -d ${includedir}/ignite ]; then find ${includedir}/ignite -type d -empty -delete; fi

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/common/include/ignite/common/concurrent.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/concurrent.h b/modules/platforms/cpp/common/include/ignite/common/concurrent.h
index 35c2209..ff0e54a 100644
--- a/modules/platforms/cpp/common/include/ignite/common/concurrent.h
+++ b/modules/platforms/cpp/common/include/ignite/common/concurrent.h
@@ -19,6 +19,7 @@
 #define _IGNITE_COMMON_CONCURRENT
 
 #include <cassert>
+#include <utility>
 
 #include "ignite/common/concurrent_os.h"
 
@@ -121,10 +122,15 @@ namespace ignite
             public:
                 friend class EnableSharedFromThis<T>;
 
+                template<typename T2>
+                friend class SharedPointer;
+
                 /**
                  * Constructor.
                  */
-                SharedPointer() : impl(0)
+                SharedPointer() :
+                    ptr(0),
+                    impl(0)
                 {
                     // No-op.
                 }
@@ -133,17 +139,17 @@ namespace ignite
                  * Constructor.
                  *
                  * @param ptr Raw pointer.
+                 * @param deleter Delete function.
                  */
-                explicit SharedPointer(T* ptr)
+                SharedPointer(T* ptr, void(*deleter)(T*) = &SharedPointerDefaultDeleter<T>) :
+                    ptr(ptr),
+                    impl(0)
                 {
                     if (ptr)
                     {
-                        void(*deleter)(T*) = (void(*)(T*)) &SharedPointerDefaultDeleter<T>;
                         impl = new SharedPointerImpl(ptr, reinterpret_cast<SharedPointerImpl::DeleterType>(deleter));
                         ImplEnableShared(ptr, impl);
                     }
-                    else
-                        impl = 0;
                 }
 
                 /**
@@ -152,15 +158,16 @@ namespace ignite
                  * @param ptr Raw pointer.
                  * @param deleter Delete function.
                  */
-                SharedPointer(T* ptr, void(*deleter)(T*))
+                template<typename T2>
+                SharedPointer(T2* ptr, void(*deleter)(T2*) = &SharedPointerDefaultDeleter<T2>) :
+                    ptr(ptr),
+                    impl(0)
                 {
                     if (ptr)
                     {
                         impl = new SharedPointerImpl(ptr, reinterpret_cast<SharedPointerImpl::DeleterType>(deleter));
                         ImplEnableShared(ptr, impl);
                     }
-                    else
-                        impl = 0;
                 }
 
                 /**
@@ -169,6 +176,21 @@ namespace ignite
                  * @param other Instance to copy.
                  */
                 SharedPointer(const SharedPointer& other) :
+                    ptr(other.ptr),
+                    impl(other.impl)
+                {
+                    if (impl)
+                        impl->Increment();
+                }
+
+                /**
+                 * Copy constructor.
+                 *
+                 * @param other Instance to copy.
+                 */
+                template<typename T2>
+                SharedPointer(const SharedPointer<T2>& other) :
+                    ptr(other.ptr),
                     impl(other.impl)
                 {
                     if (impl)
@@ -186,26 +208,43 @@ namespace ignite
                     {
                         SharedPointer tmp(other);
 
-                        std::swap(impl, tmp.impl);
+                        Swap(tmp);
                     }
 
                     return *this;
                 }
 
                 /**
+                 * Assignment operator.
+                 *
+                 * @param other Other instance.
+                 */
+                template<typename T2>
+                SharedPointer& operator=(const SharedPointer<T2>& other)
+                {
+                    SharedPointer<T> tmp(other);
+
+                    Swap(tmp);
+
+                    return *this;
+                }
+
+                /**
                  * Destructor.
                  */
                 ~SharedPointer()
                 {
                     if (impl && impl->Decrement())
                     {
-                        T* ptr = Get();
+                        void* ptr0 = impl->Pointer();
 
-                        void(*deleter)(T*) = reinterpret_cast<void(*)(T*)>(impl->Deleter());
+                        void(*deleter)(void*) = impl->Deleter();
 
-                        deleter(ptr);
+                        deleter(ptr0);
 
                         delete impl;
+
+                        ptr = 0;
                     }
                 }
 
@@ -216,7 +255,7 @@ namespace ignite
                  */
                 T* Get()
                 {
-                    return impl ? static_cast<T*>(impl->Pointer()) : 0;
+                    return ptr;
                 }
 
                 /**
@@ -226,7 +265,7 @@ namespace ignite
                  */
                 const T* Get() const
                 {
-                    return impl ? static_cast<T*>(impl->Pointer()) : 0;
+                    return ptr;
                 }
 
                 /**
@@ -245,7 +284,30 @@ namespace ignite
                     return impl != 0;
                 }
 
+                /**
+                 * Swap pointer content with another instance.
+                 *
+                 * @param other Other instance.
+                 */
+                void Swap(SharedPointer& other)
+                {
+                    if (this != &other)
+                    {
+                        T* ptrTmp = ptr;
+                        SharedPointerImpl* implTmp = impl;
+
+                        ptr = other.ptr;
+                        impl = other.impl;
+
+                        other.ptr = ptrTmp;
+                        other.impl = implTmp;
+                    }
+                }
+
             private:
+                /* Pointer. */
+                T* ptr;
+
                 /** Implementation. */
                 SharedPointerImpl* impl;
             };

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/common/include/ignite/common/reference_impl.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/reference_impl.h b/modules/platforms/cpp/common/include/ignite/common/reference_impl.h
new file mode 100644
index 0000000..e38da8c
--- /dev/null
+++ b/modules/platforms/cpp/common/include/ignite/common/reference_impl.h
@@ -0,0 +1,286 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::ReferenceImplBase class and its implementations.
+ */
+
+#ifndef _IGNITE_COMMON_REFERENCE_IMPL
+#define _IGNITE_COMMON_REFERENCE_IMPL
+
+#include <utility>
+
+#include <ignite/common/common.h>
+
+namespace ignite
+{
+    namespace common
+    {
+        // Any number is good as long as it is not null.
+        enum { POINTER_CAST_MAGIC_NUMBER = 80000 };
+
+        /**
+         * Interface for constant Reference implementation class template.
+         */
+        class ConstReferenceImplBase
+        {
+        public:
+            /**
+             * Destructor.
+             */
+            virtual ~ConstReferenceImplBase()
+            {
+                // No-op.
+            }
+
+            /**
+             * Get the constant pointer.
+             *
+             * @return Constant pointer to underlying value.
+             */
+            virtual const void* Get() const = 0;
+        };
+
+        /**
+         * Interface for Reference implementation class template.
+         */
+        class ReferenceImplBase : public ConstReferenceImplBase
+        {
+        public:
+            /**
+             * Destructor.
+             */
+            virtual ~ReferenceImplBase()
+            {
+                // No-op.
+            }
+
+            virtual const void* Get() const = 0;
+
+            /**
+             * Get the pointer.
+             *
+             * @return Pointer to underlying value.
+             */
+            virtual void* Get() = 0;
+        };
+
+        /**
+         * Reference class implementation for smart pointers.
+         *
+         * Note, this class does not implement any smart pointer functionality
+         * itself, instead it wraps one of the existing wide-spread smart
+         * pointer implementations and provides unified interface for them.
+         */
+        template<typename P>
+        class ReferenceSmartPointer : public ReferenceImplBase
+        {
+        public:
+            /**
+             * Destructor.
+             */
+            virtual ~ReferenceSmartPointer()
+            {
+                // No-op.
+            }
+
+            /**
+             * Default constructor.
+             */
+            ReferenceSmartPointer() :
+                ptr()
+            {
+                // No-op.
+            }
+
+            virtual const void* Get() const
+            {
+                return reinterpret_cast<const void*>(&(*ptr));
+            }
+
+            virtual void* Get()
+            {
+                return reinterpret_cast<void*>(&(*ptr));
+            }
+
+            /**
+             * Swap underlying smart pointer.
+             *
+             * @param other Another instance.
+             */
+            void Swap(P& other)
+            {
+                using std::swap;
+
+                swap(ptr, other);
+            }
+
+        private:
+            /** Underlying pointer. */
+            P ptr;
+        };
+
+        /**
+         * Reference implementation for the owning raw pointer.
+         */
+        template<typename T>
+        class ReferenceOwningRawPointer : public ReferenceImplBase
+        {
+        public:
+            /**
+             * Destructor.
+             */
+            virtual ~ReferenceOwningRawPointer()
+            {
+                delete ptr;
+            }
+
+            /**
+             * Default constructor.
+             */
+            ReferenceOwningRawPointer() :
+                ptr(0)
+            {
+                // No-op.
+            }
+
+            /**
+             * Pointer constructor.
+             *
+             * @param ptr Pointer to take ownership over.
+             */
+            ReferenceOwningRawPointer(T* ptr) :
+                ptr(ptr)
+            {
+                // No-op.
+            }
+
+            virtual const void* Get() const
+            {
+                return reinterpret_cast<const void*>(ptr);
+            }
+
+            virtual void* Get()
+            {
+                return reinterpret_cast<void*>(ptr);
+            }
+
+        private:
+            /** Underlying pointer. */
+            T* ptr;
+        };
+
+        /**
+         * Reference implementation for the raw pointer.
+         */
+        template<typename T>
+        class ReferenceNonOwningRawPointer : public ReferenceImplBase
+        {
+        public:
+            /**
+             * Destructor.
+             */
+            virtual ~ReferenceNonOwningRawPointer()
+            {
+                // No-op.
+            }
+
+            /**
+             * Default constructor.
+             */
+            ReferenceNonOwningRawPointer() :
+                ptr(0)
+            {
+                // No-op.
+            }
+
+            /**
+             * Pointer constructor.
+             *
+             * @param ptr Pointer.
+             */
+            ReferenceNonOwningRawPointer(T* ptr) :
+                ptr(ptr)
+            {
+                // No-op.
+            }
+
+            virtual const void* Get() const
+            {
+                return reinterpret_cast<const void*>(ptr);
+            }
+
+            virtual void* Get()
+            {
+                return reinterpret_cast<void*>(ptr);
+            }
+
+        private:
+            /** Underlying pointer. */
+            T* ptr;
+        };
+
+        /**
+         * Constant reference implementation for the raw pointer.
+         */
+        template<typename T>
+        class ConstReferenceNonOwningRawPointer : public ConstReferenceImplBase
+        {
+        public:
+            /**
+             * Destructor.
+             */
+            virtual ~ConstReferenceNonOwningRawPointer()
+            {
+                // No-op.
+            }
+
+            /**
+             * Default constructor.
+             */
+            ConstReferenceNonOwningRawPointer() :
+                ptr(0)
+            {
+                // No-op.
+            }
+
+            /**
+             * Pointer constructor.
+             *
+             * @param ptr Pointer.
+             */
+            ConstReferenceNonOwningRawPointer(const T* ptr) :
+                ptr(ptr)
+            {
+                // No-op.
+            }
+
+            virtual const void* Get() const
+            {
+                return reinterpret_cast<const void*>(ptr);
+            }
+
+        private:
+            /** Underlying pointer. */
+            const T* ptr;
+        };
+
+    }
+}
+
+#endif //_IGNITE_COMMON_REFERENCE_IMPL
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/common/include/ignite/reference.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/reference.h b/modules/platforms/cpp/common/include/ignite/reference.h
new file mode 100644
index 0000000..b026ad7
--- /dev/null
+++ b/modules/platforms/cpp/common/include/ignite/reference.h
@@ -0,0 +1,564 @@
+/*
+ * 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.
+ */
+
+/**
+ * @file
+ * Declares ignite::Reference class.
+ */
+
+#ifndef _IGNITE_COMMON_REFERENCE
+#define _IGNITE_COMMON_REFERENCE
+
+#include <cstddef>
+
+#include <ignite/common/common.h>
+#include <ignite/common/concurrent.h>
+#include <ignite/common/reference_impl.h>
+
+namespace ignite
+{
+    template<typename T>
+    class Reference;
+
+    /**
+     * Constant Reference class.
+     *
+     * Abstraction on any reference-type object, from simple raw pointers and
+     * references to standard library smart pointers. Provides only constant
+     * access to the underlying data.
+     *
+     * There are no requirements for the template type T.
+     */
+    template<typename T>
+    class ConstReference
+    {
+        template<typename>
+        friend class ConstReference;
+
+        template<typename>
+        friend class Reference;
+
+    public:
+        /**
+         * Default constructor.
+         */
+        ConstReference() :
+            ptr(),
+            offset(0)
+        {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param ptr ConstReference class implementation.
+         * @param offset Pointer offset.
+         */
+        explicit ConstReference(common::ConstReferenceImplBase* ptr, ptrdiff_t offset = 0) :
+            ptr(ptr),
+            offset(offset)
+        {
+            // No-op.
+        }
+
+        /**
+         * Copy constructor.
+         *
+         * @param other Another instance.
+         */
+        ConstReference(const ConstReference& other) :
+            ptr(other.ptr),
+            offset(other.offset)
+        {
+            // No-op.
+        }
+
+        /**
+         * Copy constructor.
+         *
+         * Constant reference of type T2 should be static-castable to constant
+         * reference of type T.
+         *
+         * @param other Another instance.
+         */
+        template<typename T2>
+        ConstReference(const ConstReference<T2>& other) :
+            ptr(other.ptr),
+            offset(other.offset)
+        {
+            T2* p0 = reinterpret_cast<T2*>(common::POINTER_CAST_MAGIC_NUMBER);
+            T* p1 = static_cast<T*>(p0);
+
+            ptrdiff_t diff = reinterpret_cast<ptrdiff_t>(p1) - reinterpret_cast<ptrdiff_t>(p0);
+            offset += diff;
+        }
+
+        /**
+         * Assignment operator.
+         *
+         * @param other Another instance.
+         */
+        ConstReference& operator=(const ConstReference& other)
+        {
+            ptr = other.ptr;
+            offset = other.offset;
+
+            return *this;
+        }
+        
+        /**
+         * Assignment operator.
+         *
+         * Constant reference of type T2 should be static-castable to constant
+         * reference of type T.
+         *
+         * @param other Another instance.
+         */
+        template<typename T2>
+        ConstReference& operator=(const ConstReference<T2>& other)
+        {
+            ptr = other.ptr;
+            offset = other.offset;
+
+            T2* p0 = reinterpret_cast<T2*>(common::POINTER_CAST_MAGIC_NUMBER);
+            T* p1 = static_cast<T*>(p0);
+
+            ptrdiff_t diff = reinterpret_cast<ptrdiff_t>(p1) - reinterpret_cast<ptrdiff_t>(p0);
+            offset += diff;
+
+            return *this;
+        }
+
+        /**
+         * Destructor.
+         */
+        ~ConstReference()
+        {
+            // No-op.
+        }
+
+        /**
+         * Dereference the pointer.
+         *
+         * If the pointer is null then this operation causes undefined
+         * behaviour.
+         *
+         * @return Constant reference to underlying value.
+         */
+        const T& Get() const
+        {
+            return *reinterpret_cast<const T*>(reinterpret_cast<ptrdiff_t>(ptr.Get()->Get()) + offset);
+        }
+
+        /**
+         * Check if the pointer is null.
+         *
+         * @return True if the value is null.
+         */
+        bool IsNull() const
+        {
+            const common::ConstReferenceImplBase* raw = ptr.Get();
+
+            return !raw || !raw->Get();
+        }
+
+    private:
+        /** Implementation. */
+        common::concurrent::SharedPointer<common::ConstReferenceImplBase> ptr;
+
+        /** Address offset. */
+        ptrdiff_t offset;
+    };
+
+    /**
+     * Reference class.
+     *
+     * Abstraction on any reference-type object, from simple raw pointers and
+     * references to standard library smart pointers.
+     *
+     * There are no requirements for the template type T.
+     */
+    template<typename T>
+    class Reference
+    {
+        template<typename>
+        friend class Reference;
+    public:
+        /**
+         * Default constructor.
+         */
+        Reference() :
+            ptr(),
+            offset(0)
+        {
+            // No-op.
+        }
+
+        /**
+         * Constructor.
+         *
+         * @param ptr Reference class implementation.
+         * @param offset Pointer offset.
+         */
+        explicit Reference(common::ReferenceImplBase* ptr, ptrdiff_t offset = 0) :
+            ptr(ptr),
+            offset(offset)
+        {
+            // No-op.
+        }
+
+        /**
+         * Copy constructor.
+         *
+         * @param other Another instance.
+         */
+        Reference(const Reference& other) :
+            ptr(other.ptr),
+            offset(other.offset)
+        {
+            // No-op.
+        }
+
+        /**
+         * Copy constructor.
+         * 
+         * Reference of type T2 should be static-castable to reference of type T.
+         *
+         * @param other Another instance.
+         */
+        template<typename T2>
+        Reference(const Reference<T2>& other) :
+            ptr(other.ptr),
+            offset(other.offset)
+        {
+            T2* p0 = reinterpret_cast<T2*>(common::POINTER_CAST_MAGIC_NUMBER);
+            T* p1 = static_cast<T*>(p0);
+
+            ptrdiff_t diff = reinterpret_cast<ptrdiff_t>(p1) - reinterpret_cast<ptrdiff_t>(p0);
+            offset += diff;
+        }
+
+        /**
+         * Assignment operator.
+         *
+         * @param other Another instance.
+         */
+        Reference& operator=(const Reference& other)
+        {
+            ptr = other.ptr;
+            offset = other.offset;
+
+            return *this;
+        }
+        
+        /**
+         * Assignment operator.
+         *
+         * Reference of type T2 should be static-castable to reference of type T.
+         *
+         * @param other Another instance.
+         */
+        template<typename T2>
+        Reference& operator=(const Reference<T2>& other)
+        {
+            ptr = other.ptr;
+            offset = other.offset;
+
+            T2* p0 = reinterpret_cast<T2*>(common::POINTER_CAST_MAGIC_NUMBER);
+            T* p1 = static_cast<T*>(p0);
+
+            ptrdiff_t diff = reinterpret_cast<ptrdiff_t>(p1) - reinterpret_cast<ptrdiff_t>(p0);
+            offset += diff;
+
+            return *this;
+        }
+
+        /**
+         * Destructor.
+         */
+        ~Reference()
+        {
+            // No-op.
+        }
+
+        /**
+         * Const cast operator.
+         *
+         * Reference of type T2 should be static-castable to reference of type T.
+         *
+         * Casts this instance to constant reference.
+         */
+        template<typename T2>
+        operator ConstReference<T2>()
+        {
+            ConstReference<T2> cr;
+
+            cr.ptr = ptr;
+            cr.offset = offset;
+
+            T2* p0 = reinterpret_cast<T2*>(common::POINTER_CAST_MAGIC_NUMBER);
+            const T* p1 = static_cast<T*>(p0);
+
+            ptrdiff_t diff = reinterpret_cast<ptrdiff_t>(p1) - reinterpret_cast<ptrdiff_t>(p0);
+            cr.offset -= diff;
+
+            return cr;
+        }
+
+        /**
+         * Dereference the pointer.
+         *
+         * If the pointer is null then this operation causes undefined
+         * behaviour.
+         *
+         * @return Constant reference to underlying value.
+         */
+        const T& Get() const
+        {
+            return *reinterpret_cast<const T*>(reinterpret_cast<ptrdiff_t>(ptr.Get()->Get()) + offset);
+        }
+
+        /**
+         * Dereference the pointer.
+         *
+         * If the pointer is null then this operation causes undefined
+         * behaviour.
+         *
+         * @return Reference to underlying value.
+         */
+        T& Get()
+        {
+            return *reinterpret_cast<T*>(reinterpret_cast<ptrdiff_t>(ptr.Get()->Get()) + offset);
+        }
+
+        /**
+         * Check if the pointer is null.
+         *
+         * @return True if the value is null.
+         */
+        bool IsNull() const
+        {
+            const common::ReferenceImplBase* raw = ptr.Get();
+
+            return !raw || !raw->Get();
+        }
+
+    private:
+        /** Implementation. */
+        common::concurrent::SharedPointer<common::ReferenceImplBase> ptr;
+
+        /** Address offset. */
+        ptrdiff_t offset;
+    };
+
+    /**
+     * Make ignite::Reference instance out of smart pointer.
+     *
+     * Template type 'T' should be a smart pointer and provide pointer semantics:
+     * - There should be defined type 'T::element_type', showing underlying type.
+     * - Type 'T' should be dereferencible (should have operators
+     *   T::element_type& operator*() and const T::element_type& operator*() const).
+     * - Operation std::swap should result in valid result if applied to two
+     *   instances of that type.
+     *
+     * @param ptr Pointer.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    Reference<typename T::element_type> MakeReferenceFromSmartPointer(T ptr)
+    {
+        common::ReferenceSmartPointer<T>* impl = new common::ReferenceSmartPointer<T>();
+
+        Reference<typename T::element_type> res(impl);
+
+        impl->Swap(ptr);
+
+        return res;
+    }
+
+    /**
+     * Make ignite::ConstReference instance out of smart pointer.
+     *
+     * Template type 'T' should be a smart pointer and provide pointer semantics:
+     * - There should be defined type 'T::element_type', showing underlying type.
+     * - Type 'T' should be dereferencible (should have operators
+     *   T::element_type& operator*() and const T::element_type& operator*() const).
+     * - Operation std::swap should result in valid result if applied to two
+     *   instances of that type.
+     *
+     * @param ptr Pointer.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    ConstReference<typename T::element_type> MakeConstReferenceFromSmartPointer(T ptr)
+    {
+        common::ReferenceSmartPointer<T>* impl = new common::ReferenceSmartPointer<T>();
+
+        ConstReference<typename T::element_type> res(impl);
+
+        impl->Swap(ptr);
+
+        return res;
+    }
+
+    /**
+     * Copy object and wrap it to make ignite::Reference instance.
+     *
+     * Template type 'T' should be copy-constructible.
+     *
+     * @param val Instance.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    Reference<T> MakeReferenceFromCopy(const T& val)
+    {
+        common::ReferenceOwningRawPointer<T>* impl = new common::ReferenceOwningRawPointer<T>(new T(val));
+
+        return Reference<T>(impl);
+    }
+
+    /**
+     * Copy object and wrap it to make ignite::ConstReference instance.
+     *
+     * Template type 'T' should be copy-constructible.
+     *
+     * @param val Instance.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    ConstReference<T> MakeConstReferenceFromCopy(const T& val)
+    {
+        common::ReferenceOwningRawPointer<T>* impl = new common::ReferenceOwningRawPointer<T>(new T(val));
+
+        return ConstReference<T>(impl);
+    }
+
+    /**
+     * Make ignite::Reference instance out of pointer and pass its ownership.
+     * Passed object deleted by Ignite when no longer needed.
+     *
+     * There are no requirements for the template type T.
+     *
+     * @param val Instance.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    Reference<T> MakeReferenceFromOwningPointer(T* val)
+    {
+        common::ReferenceOwningRawPointer<T>* impl = new common::ReferenceOwningRawPointer<T>(val);
+
+        return Reference<T>(impl);
+    }
+
+    /**
+     * Make ignite::ConstReference instance out of pointer and pass its ownership.
+     * Passed object deleted by Ignite when no longer needed.
+     *
+     * There are no requirements for the template type T.
+     *
+     * @param val Instance.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    ConstReference<T> MakeConstReferenceFromOwningPointer(T* val)
+    {
+        common::ReferenceOwningRawPointer<T>* impl = new common::ReferenceOwningRawPointer<T>(val);
+
+        return ConstReference<T>(impl);
+    }
+
+    /**
+     * Make ignite::Reference instance out of reference.
+     * Ignite do not manage passed object and does not affect its lifetime.
+     *
+     * There are no requirements for the template type T.
+     *
+     * @param val Reference.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    Reference<T> MakeReference(T& val)
+    {
+        common::ReferenceNonOwningRawPointer<T>* impl = new common::ReferenceNonOwningRawPointer<T>(&val);
+
+        return Reference<T>(impl);
+    }
+
+    /**
+     * Make ignite::Reference instance out of pointer.
+     * Ignite do not manage passed object and does not affect its lifetime.
+     *
+     * There are no requirements for the template type T.
+     *
+     * @param val Reference.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    Reference<T> MakeReference(T* val)
+    {
+        common::ReferenceNonOwningRawPointer<T>* impl = new common::ReferenceNonOwningRawPointer<T>(val);
+
+        return Reference<T>(impl);
+    }
+
+    /**
+     * Make ignite::ConstReference instance out of constant reference.
+     * Ignite do not manage passed object and does not affect its lifetime.
+     *
+     * There are no requirements for the template type T.
+     *
+     * @param val Reference.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    ConstReference<T> MakeConstReference(const T& val)
+    {
+        common::ConstReferenceNonOwningRawPointer<T>* impl = new common::ConstReferenceNonOwningRawPointer<T>(&val);
+
+        return ConstReference<T>(impl);
+    }
+
+    /**
+     * Make ignite::ConstReference instance out of constant pointer.
+     * Ignite do not manage passed object and does not affect its lifetime.
+     *
+     * There are no requirements for the template type T.
+     *
+     * @param val Reference.
+     * @return Implementation defined value. User should not explicitly use the
+     *     returned value.
+     */
+    template<typename T>
+    ConstReference<T> MakeConstReference(const T* val)
+    {
+        common::ConstReferenceNonOwningRawPointer<T>* impl = new common::ConstReferenceNonOwningRawPointer<T>(val);
+
+        return ConstReference<T>(impl);
+    }
+}
+
+#endif //_IGNITE_COMMON_REFERENCE
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/common/project/vs/common.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/project/vs/common.vcxproj b/modules/platforms/cpp/common/project/vs/common.vcxproj
index 149fa48..99fd551 100644
--- a/modules/platforms/cpp/common/project/vs/common.vcxproj
+++ b/modules/platforms/cpp/common/project/vs/common.vcxproj
@@ -171,10 +171,12 @@
     <ClInclude Include="..\..\include\ignite\common\dynamic_size_array.h" />
     <ClInclude Include="..\..\include\ignite\common\fixed_size_array.h" />
     <ClInclude Include="..\..\include\ignite\common\bits.h" />
+    <ClInclude Include="..\..\include\ignite\common\reference_impl.h" />
     <ClInclude Include="..\..\include\ignite\common\utils.h" />
     <ClInclude Include="..\..\include\ignite\date.h" />
     <ClInclude Include="..\..\include\ignite\guid.h" />
     <ClInclude Include="..\..\include\ignite\ignite_error.h" />
+    <ClInclude Include="..\..\include\ignite\reference.h" />
     <ClInclude Include="..\..\include\ignite\timestamp.h" />
     <ClInclude Include="..\..\os\win\include\ignite\common\common.h" />
     <ClInclude Include="..\..\os\win\include\ignite\common\concurrent_os.h" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/common/project/vs/common.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/project/vs/common.vcxproj.filters b/modules/platforms/cpp/common/project/vs/common.vcxproj.filters
index ae17daf..d99722b 100644
--- a/modules/platforms/cpp/common/project/vs/common.vcxproj.filters
+++ b/modules/platforms/cpp/common/project/vs/common.vcxproj.filters
@@ -58,6 +58,12 @@
     <ClInclude Include="..\..\include\ignite\common\decimal.h">
       <Filter>Code\common</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\include\ignite\common\reference_impl.h">
+      <Filter>Code\common</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\reference.h">
+      <Filter>Code</Filter>
+    </ClInclude>
   </ItemGroup>
   <ItemGroup>
     <ClCompile Include="..\..\src\date.cpp">

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core-test/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/Makefile.am b/modules/platforms/cpp/core-test/Makefile.am
index 42680cd..3cf1d0e 100644
--- a/modules/platforms/cpp/core-test/Makefile.am
+++ b/modules/platforms/cpp/core-test/Makefile.am
@@ -43,15 +43,20 @@ AM_CXXFLAGS = \
 
 ignite_tests_LDADD = \
     @top_srcdir@/core/libignite.la \
-    -lpthread
+    -lpthread \
+    -lboost_thread \
+    -lboost_system \
+    -lboost_chrono
 
 ignite_tests_LDFLAGS = \
     -static-libtool-libs
 
 ignite_tests_SOURCES = \
+    src/reference_test.cpp \
     src/bits_test.cpp \
     src/cache_test.cpp \
     src/cache_query_test.cpp \
+    src/continuous_query_test.cpp \
     src/concurrent_test.cpp \
     src/ignition_test.cpp \
     src/interop_memory_test.cpp \

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core-test/config/cache-query-continuous.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/config/cache-query-continuous.xml b/modules/platforms/cpp/core-test/config/cache-query-continuous.xml
new file mode 100644
index 0000000..1b940fd
--- /dev/null
+++ b/modules/platforms/cpp/core-test/config/cache-query-continuous.xml
@@ -0,0 +1,87 @@
+<?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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="transactional_no_backup"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+                    <property name="backups" value="0"/>
+                    <property name="startSize" value="10"/>
+                    <property name="queryEntities">
+                        <list>
+                            <bean class="org.apache.ignite.cache.QueryEntity">
+                                <property name="keyType" value="java.lang.Integer"/>
+                                <property name="valueType" value="TestEntry"/>
+
+                                <property name="fields">
+                                    <map>
+                                        <entry key="value" value="java.lang.Integer"/>
+                                    </map>
+                                </property>
+
+                                <property name="indexes">
+                                    <list>
+                                        <bean class="org.apache.ignite.cache.QueryIndex">
+                                            <property name="fields">
+                                                <map>
+                                                    <entry key="value" value="true"/>
+                                                </map>
+                                            </property>
+                                            <property name="indexType" value="FULLTEXT"/>
+                                        </bean>
+                                    </list>
+                                </property>
+                            </bean>
+                        </list>
+                    </property>
+                </bean>
+            </list>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+                <property name="socketTimeout" value="300" />
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
index 6f13719..a41d8f8 100644
--- a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
+++ b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj
@@ -43,6 +43,7 @@
     <ClCompile Include="..\..\src\decimal_test.cpp" />
     <ClCompile Include="..\..\src\dynamic_size_array_test.cpp" />
     <ClCompile Include="..\..\src\fixed_size_array_test.cpp" />
+    <ClCompile Include="..\..\src\continuous_query_test.cpp" />
     <ClCompile Include="..\..\src\ignite_error_test.cpp" />
     <ClCompile Include="..\..\src\ignition_test.cpp" />
     <ClCompile Include="..\..\src\handle_registry_test.cpp" />
@@ -54,6 +55,7 @@
     <ClCompile Include="..\..\src\interop_memory_test.cpp" />
     <ClCompile Include="..\..\src\interop_test.cpp" />
     <ClCompile Include="..\..\src\bits_test.cpp" />
+    <ClCompile Include="..\..\src\reference_test.cpp" />
     <ClCompile Include="..\..\src\teamcity_boost.cpp" />
     <ClCompile Include="..\..\src\teamcity_messages.cpp" />
     <ClCompile Include="..\..\src\transactions_test.cpp" />
@@ -129,7 +131,7 @@
     </ClCompile>
     <Link>
       <GenerateDebugInformation>true</GenerateDebugInformation>
-      <AdditionalDependencies>$(BOOST_HOME)\lib64-msvc-10.0\libboost_unit_test_framework-vc100-mt-gd-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalDependencies>$(BOOST_HOME)\lib64-msvc-10.0\libboost_unit_test_framework-vc100-mt-gd-1_58.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_thread-vc100-mt-gd-1_58.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_system-vc100-mt-gd-1_58.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_chrono-vc100-mt-gd-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
       <SubSystem>Console</SubSystem>
     </Link>
   </ItemDefinitionGroup>
@@ -144,7 +146,7 @@
     </ClCompile>
     <Link>
       <GenerateDebugInformation>true</GenerateDebugInformation>
-      <AdditionalDependencies>$(BOOST_HOME)\lib64-msvc-10.0\libboost_unit_test_framework-vc100-mt-gd-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalDependencies>$(BOOST_HOME)\lib32-msvc-10.0\libboost_unit_test_framework-vc100-mt-gd-1_58.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_thread-vc100-mt-gd-1_58.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_system-vc100-mt-gd-1_58.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_chrono-vc100-mt-gd-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
     </Link>
   </ItemDefinitionGroup>
   <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|x64'">
@@ -162,7 +164,7 @@
       <GenerateDebugInformation>true</GenerateDebugInformation>
       <EnableCOMDATFolding>true</EnableCOMDATFolding>
       <OptimizeReferences>true</OptimizeReferences>
-      <AdditionalDependencies>$(BOOST_HOME)\lib64-msvc-10.0\libboost_unit_test_framework-vc100-mt-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalDependencies>$(BOOST_HOME)\lib64-msvc-10.0\libboost_unit_test_framework-vc100-mt-1_58.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_thread-vc100-mt-1_58.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_system-vc100-mt-1_58.lib;$(BOOST_HOME)\lib64-msvc-10.0\libboost_chrono-vc100-mt-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
     </Link>
   </ItemDefinitionGroup>
   <ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">
@@ -180,7 +182,7 @@
       <GenerateDebugInformation>true</GenerateDebugInformation>
       <EnableCOMDATFolding>true</EnableCOMDATFolding>
       <OptimizeReferences>true</OptimizeReferences>
-      <AdditionalDependencies>$(BOOST_HOME)\lib64-msvc-10.0\libboost_unit_test_framework-vc100-mt-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
+      <AdditionalDependencies>$(BOOST_HOME)\lib32-msvc-10.0\libboost_unit_test_framework-vc100-mt-1_58.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_thread-vc100-mt-1_58.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_system-vc100-mt-1_58.lib;$(BOOST_HOME)\lib32-msvc-10.0\libboost_chrono-vc100-mt-1_58.lib;%(AdditionalDependencies)</AdditionalDependencies>
     </Link>
   </ItemDefinitionGroup>
   <Import Project="$(VCTargetsPath)\Microsoft.Cpp.targets" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
index cf1aaca..a95e3a4 100644
--- a/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
+++ b/modules/platforms/cpp/core-test/project/vs/core-test.vcxproj.filters
@@ -58,6 +58,12 @@
     <ClCompile Include="..\..\src\interop_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\continuous_query_test.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
+    <ClCompile Include="..\..\src\reference_test.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\..\include\teamcity_messages.h">

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core-test/src/cache_query_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/cache_query_test.cpp b/modules/platforms/cpp/core-test/src/cache_query_test.cpp
index c1c26ac..928d29e 100644
--- a/modules/platforms/cpp/core-test/src/cache_query_test.cpp
+++ b/modules/platforms/cpp/core-test/src/cache_query_test.cpp
@@ -232,8 +232,6 @@ private:
     int32_t someVal;
 };
 
-
-
 namespace ignite
 {
     namespace binary
@@ -296,74 +294,6 @@ namespace ignite
     }
 }
 
-///**
-// * Test setup fixture.
-// */
-//struct CacheQueryTestSuiteFixture
-//{
-//    Ignite StartNode(const char* name)
-//    {
-//        IgniteConfiguration cfg;
-//
-//        cfg.jvmOpts.push_back("-Xdebug");
-//        cfg.jvmOpts.push_back("-Xnoagent");
-//        cfg.jvmOpts.push_back("-Djava.compiler=NONE");
-//        cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005");
-//        cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError");
-//
-//#ifdef IGNITE_TESTS_32
-//        cfg.jvmInitMem = 256;
-//        cfg.jvmMaxMem = 768;
-//#else
-//        cfg.jvmInitMem = 1024;
-//        cfg.jvmMaxMem = 4096;
-//#endif
-//
-//        cfg.springCfgPath.assign(getenv("IGNITE_NATIVE_TEST_CPP_CONFIG_PATH")).append("/cache-query.xml");
-//
-//        IgniteError err;
-//
-//        Ignite grid0 = Ignition::Start(cfg, name, &err);
-//
-//        if (err.GetCode() != IgniteError::IGNITE_SUCCESS)
-//            BOOST_ERROR(err.GetText());
-//
-//        return grid0;
-//    }
-//
-//
-//    /**
-//     * Constructor.
-//     */
-//    CacheQueryTestSuiteFixture() : 
-//        grid(StartNode("Node1"))
-//    {
-//        // No-op.
-//    }
-//
-//    /**
-//     * Destructor.
-//     */
-//    ~CacheQueryTestSuiteFixture()
-//    {
-//        Ignition::StopAll(true);
-//    }
-//
-//    /** Person cache accessor. */
-//    Cache<int, QueryPerson> GetPersonCache()
-//    {
-//        return grid.GetCache<int, QueryPerson>("QueryPerson");
-//    }
-//
-    ///** Relation cache accessor. */
-    //Cache<int, QueryRelation> GetRelationCache()
-    //{
-    //    return grid.GetCache<int, QueryRelation>("QueryRelation");
-    //}
-//
-//    /** Node started during the test. */
-//    Ignite grid;
-//};
 
 /**
  * Count number of records returned by cursor.
@@ -677,7 +607,11 @@ struct CacheQueryTestSuiteFixture
         cfg.jvmMaxMem = 4096;
 #endif
 
-        cfg.springCfgPath.assign(getenv("IGNITE_NATIVE_TEST_CPP_CONFIG_PATH")).append("/cache-query.xml");
+        const char* cfgPath = getenv("IGNITE_NATIVE_TEST_CPP_CONFIG_PATH");
+
+        BOOST_CHECK(cfgPath != 0);
+
+        cfg.springCfgPath.assign(cfgPath).append("/cache-query.xml");
 
         IgniteError err;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core-test/src/continuous_query_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/continuous_query_test.cpp b/modules/platforms/cpp/core-test/src/continuous_query_test.cpp
new file mode 100644
index 0000000..e9d7e8a
--- /dev/null
+++ b/modules/platforms/cpp/core-test/src/continuous_query_test.cpp
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _MSC_VER
+    #define BOOST_TEST_DYN_LINK
+#endif
+
+#include <deque>
+
+#include <boost/test/unit_test.hpp>
+#include <boost/optional.hpp>
+#include <boost/thread/mutex.hpp>
+#include <boost/thread/condition_variable.hpp>
+
+#include "ignite/ignition.h"
+#include "ignite/cache/cache.h"
+
+using namespace ignite;
+using namespace ignite::cache;
+using namespace ignite::cache::event;
+using namespace ignite::cache::query;
+using namespace ignite::cache::query::continuous;
+using namespace boost::unit_test;
+
+/**
+ * Very simple concurrent queue implementation.
+ */
+template<typename T>
+class ConcurrentQueue
+{
+public:
+    /*
+     * Constructor.
+     */
+    ConcurrentQueue()
+    {
+        // No-op.
+    }
+
+    /*
+     * Push next element to queue.
+     *
+     * @param val Value to push.
+     */
+    void Push(const T& val)
+    {
+        boost::unique_lock<boost::mutex> guard(mutex);
+
+        queue.push_back(val);
+
+        cv.notify_one();
+    }
+
+    /*
+     * Pull element from the queue with the specified timeout.
+     *
+     * @param val Value is placed there on success.
+     * @param timeout Timeout.
+     * @return True on success and false on timeout.
+     */
+    template <typename Rep, typename Period>
+    bool Pull(T& val, const boost::chrono::duration<Rep, Period>& timeout)
+    {
+        boost::unique_lock<boost::mutex> guard(mutex);
+
+        if (queue.empty())
+        {
+            boost::cv_status res = cv.wait_for(guard, timeout);
+
+            if (res == boost::cv_status::timeout)
+                return false;
+        }
+
+        assert(!queue.empty());
+
+        val = queue.front();
+
+        queue.pop_front();
+
+        return true;
+    }
+
+private:
+    boost::mutex mutex;
+
+    boost::condition_variable cv;
+
+    std::deque<T> queue;
+};
+
+/*
+ * Test listener class. Stores events it has been notified about in concurrent
+ * queue so they can be checked later.
+ */
+template<typename K, typename V>
+class Listener : public CacheEntryEventListener<K, V>
+{
+public:
+    /*
+     * Default constructor.
+     */
+    Listener()
+    {
+        // No-op.
+    }
+
+    /**
+     * Event callback.
+     *
+     * @param evts Events.
+     * @param num Events number.
+     */
+    virtual void OnEvent(const CacheEntryEvent<K, V>* evts, uint32_t num)
+    {
+        for (uint32_t i = 0; i < num; ++i)
+            eventQueue.Push(evts[i]);
+    }
+
+    /*
+     * Check that next received event contains specific values.
+     *
+     * @param key Key.
+     * @param oldVal Old value.
+     * @param val Current value.
+     */
+    void CheckNextEvent(const K& key, boost::optional<V> oldVal, boost::optional<V> val)
+    {
+        CacheEntryEvent<K, V> event;
+        bool success = eventQueue.Pull(event, boost::chrono::seconds(1));
+
+        BOOST_REQUIRE(success);
+
+        BOOST_CHECK_EQUAL(event.GetKey(), key);
+        BOOST_CHECK_EQUAL(event.HasOldValue(), oldVal.is_initialized());
+        BOOST_CHECK_EQUAL(event.HasValue(), val.is_initialized());
+
+        if (oldVal && event.HasOldValue())
+            BOOST_CHECK_EQUAL(event.GetOldValue().value, oldVal->value);
+
+        if (val && event.HasValue())
+            BOOST_CHECK_EQUAL(event.GetValue().value, val->value);
+    }
+
+    /*
+     * Check that there is no event for the specified ammount of time.
+     *
+     * @param timeout Timeout.
+     */
+    template <typename Rep, typename Period>
+    void CheckNoEvent(const boost::chrono::duration<Rep, Period>& timeout)
+    {
+        CacheEntryEvent<K, V> event;
+        bool success = eventQueue.Pull(event, timeout);
+
+        BOOST_REQUIRE(!success);
+    }
+
+private:
+    // Events queue.
+    ConcurrentQueue< CacheEntryEvent<K, V> > eventQueue;
+};
+
+/*
+ * Test entry.
+ */
+struct TestEntry
+{
+    /*
+     * Default constructor.
+     */
+    TestEntry() : value(0)
+    {
+        // No-op.
+    }
+
+    /*
+     * Constructor.
+     */
+    TestEntry(int32_t val) : value(val)
+    {
+        // No-op.
+    }
+
+    /* Value */
+    int32_t value;
+};
+
+namespace ignite
+{
+    namespace binary
+    {
+        /**
+        * Binary type definition.
+        */
+        IGNITE_BINARY_TYPE_START(TestEntry)
+            IGNITE_BINARY_GET_TYPE_ID_AS_HASH(TestEntry)
+            IGNITE_BINARY_GET_TYPE_NAME_AS_IS(TestEntry)
+            IGNITE_BINARY_GET_FIELD_ID_AS_HASH
+            IGNITE_BINARY_GET_HASH_CODE_ZERO(TestEntry)
+            IGNITE_BINARY_IS_NULL_FALSE(TestEntry)
+            IGNITE_BINARY_GET_NULL_DEFAULT_CTOR(TestEntry)
+
+            void Write(BinaryWriter& writer, const TestEntry& obj)
+            {
+                writer.WriteInt32("value", obj.value);
+            }
+
+            TestEntry Read(BinaryReader& reader)
+            {
+                TestEntry res;
+                res.value = reader.ReadInt32("value");
+
+                return res;
+            }
+
+        IGNITE_BINARY_TYPE_END
+    }
+}
+
+/*
+ * Test setup fixture.
+ */
+struct ContinuousQueryTestSuiteFixture
+{
+    Ignite grid;
+
+    Cache<int, TestEntry> cache;
+
+    /*
+     * Get configuration for nodes.
+     */
+    IgniteConfiguration GetConfiguration()
+    {
+        IgniteConfiguration cfg;
+
+        cfg.jvmOpts.push_back("-Xdebug");
+        cfg.jvmOpts.push_back("-Xnoagent");
+        cfg.jvmOpts.push_back("-Djava.compiler=NONE");
+        cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005");
+        cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError");
+
+#ifdef IGNITE_TESTS_32
+        cfg.jvmInitMem = 256;
+        cfg.jvmMaxMem = 768;
+#else
+        cfg.jvmInitMem = 1024;
+        cfg.jvmMaxMem = 4096;
+#endif
+
+        char* cfgPath = getenv("IGNITE_NATIVE_TEST_CPP_CONFIG_PATH");
+
+        cfg.springCfgPath = std::string(cfgPath).append("/").append("cache-query-continuous.xml");
+
+        return cfg;
+    }
+
+    /*
+     * Constructor.
+     */
+    ContinuousQueryTestSuiteFixture() :
+        grid(Ignition::Start(GetConfiguration(), "node-01")),
+        cache(grid.GetCache<int, TestEntry>("transactional_no_backup"))
+    {
+        // No-op.
+    }
+
+    /*
+     * Destructor.
+     */
+    ~ContinuousQueryTestSuiteFixture()
+    {
+        Ignition::StopAll(false);
+
+        grid = Ignite();
+    }
+};
+
+void CheckEvents(Cache<int, TestEntry>& cache, Listener<int, TestEntry>& lsnr)
+{
+    cache.Put(1, TestEntry(10));
+    lsnr.CheckNextEvent(1, boost::none, TestEntry(10));
+
+    cache.Put(1, TestEntry(20));
+    lsnr.CheckNextEvent(1, TestEntry(10), TestEntry(20));
+
+    cache.Put(2, TestEntry(20));
+    lsnr.CheckNextEvent(2, boost::none, TestEntry(20));
+
+    cache.Remove(1);
+    lsnr.CheckNextEvent(1, TestEntry(20), boost::none);
+}
+
+BOOST_FIXTURE_TEST_SUITE(ContinuousQueryTestSuite, ContinuousQueryTestSuiteFixture)
+
+BOOST_AUTO_TEST_CASE(TestBasic)
+{
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestInitialQueryScan)
+{
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    cache.Put(11, TestEntry(111));
+    cache.Put(22, TestEntry(222));
+    cache.Put(33, TestEntry(333));
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry, ScanQuery());
+
+    std::vector< CacheEntry<int, TestEntry> > vals;
+
+    handle.GetInitialQueryCursor().GetAll(vals);
+
+    BOOST_CHECK_THROW(handle.GetInitialQueryCursor(), IgniteError);
+
+    BOOST_REQUIRE_EQUAL(vals.size(), 3);
+
+    BOOST_CHECK_EQUAL(vals[0].GetKey(), 11);
+    BOOST_CHECK_EQUAL(vals[1].GetKey(), 22);
+    BOOST_CHECK_EQUAL(vals[2].GetKey(), 33);
+
+    BOOST_CHECK_EQUAL(vals[0].GetValue().value, 111);
+    BOOST_CHECK_EQUAL(vals[1].GetValue().value, 222);
+    BOOST_CHECK_EQUAL(vals[2].GetValue().value, 333);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestInitialQuerySql)
+{
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    cache.Put(11, TestEntry(111));
+    cache.Put(22, TestEntry(222));
+    cache.Put(33, TestEntry(333));
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry, SqlQuery("TestEntry", "value > 200"));
+
+    std::vector< CacheEntry<int, TestEntry> > vals;
+
+    handle.GetInitialQueryCursor().GetAll(vals);
+
+    BOOST_CHECK_THROW(handle.GetInitialQueryCursor(), IgniteError);
+
+    BOOST_REQUIRE_EQUAL(vals.size(), 2);
+
+    BOOST_CHECK_EQUAL(vals[0].GetKey(), 22);
+    BOOST_CHECK_EQUAL(vals[1].GetKey(), 33);
+
+    BOOST_CHECK_EQUAL(vals[0].GetValue().value, 222);
+    BOOST_CHECK_EQUAL(vals[1].GetValue().value, 333);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestInitialQueryText)
+{
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    cache.Put(11, TestEntry(111));
+    cache.Put(22, TestEntry(222));
+    cache.Put(33, TestEntry(333));
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry, TextQuery("TestEntry", "222"));
+
+    std::vector< CacheEntry<int, TestEntry> > vals;
+
+    handle.GetInitialQueryCursor().GetAll(vals);
+
+    BOOST_CHECK_THROW(handle.GetInitialQueryCursor(), IgniteError);
+
+    BOOST_REQUIRE_EQUAL(vals.size(), 1);
+
+    BOOST_CHECK_EQUAL(vals[0].GetKey(), 22);
+
+    BOOST_CHECK_EQUAL(vals[0].GetValue().value, 222);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestBasicNoExcept)
+{
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    IgniteError err;
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry, err);
+
+    BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_SUCCESS);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestInitialQueryScanNoExcept)
+{
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    cache.Put(11, TestEntry(111));
+    cache.Put(22, TestEntry(222));
+    cache.Put(33, TestEntry(333));
+
+    IgniteError err;
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry, ScanQuery(), err);
+
+    BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_SUCCESS);
+
+    std::vector< CacheEntry<int, TestEntry> > vals;
+
+    handle.GetInitialQueryCursor().GetAll(vals);
+
+    BOOST_CHECK_THROW(handle.GetInitialQueryCursor(), IgniteError);
+
+    BOOST_REQUIRE_EQUAL(vals.size(), 3);
+
+    BOOST_CHECK_EQUAL(vals[0].GetKey(), 11);
+    BOOST_CHECK_EQUAL(vals[1].GetKey(), 22);
+    BOOST_CHECK_EQUAL(vals[2].GetKey(), 33);
+
+    BOOST_CHECK_EQUAL(vals[0].GetValue().value, 111);
+    BOOST_CHECK_EQUAL(vals[1].GetValue().value, 222);
+    BOOST_CHECK_EQUAL(vals[2].GetValue().value, 333);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestInitialQuerySqlNoExcept)
+{
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    cache.Put(11, TestEntry(111));
+    cache.Put(22, TestEntry(222));
+    cache.Put(33, TestEntry(333));
+
+    IgniteError err;
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry, SqlQuery("TestEntry", "value > 200"), err);
+
+    BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_SUCCESS);
+
+    std::vector< CacheEntry<int, TestEntry> > vals;
+
+    handle.GetInitialQueryCursor().GetAll(vals);
+
+    BOOST_CHECK_THROW(handle.GetInitialQueryCursor(), IgniteError);
+
+    BOOST_REQUIRE_EQUAL(vals.size(), 2);
+
+    BOOST_CHECK_EQUAL(vals[0].GetKey(), 22);
+    BOOST_CHECK_EQUAL(vals[1].GetKey(), 33);
+
+    BOOST_CHECK_EQUAL(vals[0].GetValue().value, 222);
+    BOOST_CHECK_EQUAL(vals[1].GetValue().value, 333);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestInitialQueryTextNoExcept)
+{
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    cache.Put(11, TestEntry(111));
+    cache.Put(22, TestEntry(222));
+    cache.Put(33, TestEntry(333));
+
+    IgniteError err;
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry, TextQuery("TestEntry", "222"), err);
+
+    BOOST_REQUIRE(err.GetCode() == IgniteError::IGNITE_SUCCESS);
+
+    std::vector< CacheEntry<int, TestEntry> > vals;
+
+    handle.GetInitialQueryCursor().GetAll(vals);
+
+    BOOST_CHECK_THROW(handle.GetInitialQueryCursor(), IgniteError);
+
+    BOOST_REQUIRE_EQUAL(vals.size(), 1);
+
+    BOOST_CHECK_EQUAL(vals[0].GetKey(), 22);
+
+    BOOST_CHECK_EQUAL(vals[0].GetValue().value, 222);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestExpiredQuery)
+{
+    Listener<int, TestEntry> lsnr;
+    ContinuousQueryHandle<int, TestEntry> handle;
+
+    {
+        // Query scope.
+        ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+        handle = cache.QueryContinuous(qry);
+    }
+
+    // Query is destroyed here.
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestSetGetLocal)
+{
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    BOOST_CHECK(!qry.GetLocal());
+
+    qry.SetLocal(true);
+
+    BOOST_CHECK(qry.GetLocal());
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry);
+
+    BOOST_CHECK(qry.GetLocal());
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestGetSetBufferSize)
+{
+    typedef ContinuousQuery<int, TestEntry> QueryType;
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    BOOST_CHECK_EQUAL(qry.GetBufferSize(), QueryType::DEFAULT_BUFFER_SIZE);
+
+    qry.SetBufferSize(2 * QueryType::DEFAULT_BUFFER_SIZE);
+
+    BOOST_CHECK_EQUAL(qry.GetBufferSize(), 2 * QueryType::DEFAULT_BUFFER_SIZE);
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry);
+
+    BOOST_CHECK_EQUAL(qry.GetBufferSize(), 2 * QueryType::DEFAULT_BUFFER_SIZE);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestGetSetTimeInterval)
+{
+    typedef ContinuousQuery<int, TestEntry> QueryType;
+    Listener<int, TestEntry> lsnr;
+
+    ContinuousQuery<int, TestEntry> qry(MakeReference(lsnr));
+
+    qry.SetBufferSize(10);
+
+    BOOST_CHECK_EQUAL(qry.GetTimeInterval(), static_cast<int>(QueryType::DEFAULT_TIME_INTERVAL));
+
+    qry.SetTimeInterval(500);
+
+    BOOST_CHECK_EQUAL(qry.GetTimeInterval(), 500);
+
+    ContinuousQueryHandle<int, TestEntry> handle = cache.QueryContinuous(qry);
+
+    BOOST_CHECK_EQUAL(qry.GetTimeInterval(), 500);
+
+    CheckEvents(cache, lsnr);
+}
+
+BOOST_AUTO_TEST_CASE(TestPublicPrivateConstantsConsistence)
+{
+    typedef ContinuousQuery<int, TestEntry> QueryType;
+    typedef impl::cache::query::continuous::ContinuousQueryImpl<int, TestEntry> QueryImplType;
+    
+    BOOST_CHECK_EQUAL(static_cast<int>(QueryImplType::DEFAULT_TIME_INTERVAL),
+        static_cast<int>(QueryType::DEFAULT_TIME_INTERVAL));
+
+    BOOST_CHECK_EQUAL(static_cast<int>(QueryImplType::DEFAULT_BUFFER_SIZE),
+        static_cast<int>(QueryType::DEFAULT_BUFFER_SIZE));
+}
+
+BOOST_AUTO_TEST_SUITE_END()
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core-test/src/handle_registry_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/handle_registry_test.cpp b/modules/platforms/cpp/core-test/src/handle_registry_test.cpp
index bc4a654..0956d9b 100644
--- a/modules/platforms/cpp/core-test/src/handle_registry_test.cpp
+++ b/modules/platforms/cpp/core-test/src/handle_registry_test.cpp
@@ -36,7 +36,7 @@ struct HandleRegistryTestProbe
     }
 };
 
-class HandleRegistryTestEntry : public HandleRegistryEntry
+class HandleRegistryTestEntry
 {
 public:
     HandleRegistryTestEntry(HandleRegistryTestProbe* probe) : probe(probe)
@@ -67,9 +67,9 @@ BOOST_AUTO_TEST_CASE(TestCritical)
     HandleRegistryTestEntry* entry1 = new HandleRegistryTestEntry(&probe1);
     HandleRegistryTestEntry* entry2 = new HandleRegistryTestEntry(&probe2);
 
-    int64_t hnd0 = reg.AllocateCritical(SharedPointer<HandleRegistryEntry>(entry0));
-    int64_t hnd1 = reg.AllocateCritical(SharedPointer<HandleRegistryEntry>(entry1));
-    int64_t hnd2 = reg.AllocateCritical(SharedPointer<HandleRegistryEntry>(entry2));
+    int64_t hnd0 = reg.AllocateCritical(SharedPointer<HandleRegistryTestEntry>(entry0));
+    int64_t hnd1 = reg.AllocateCritical(SharedPointer<HandleRegistryTestEntry>(entry1));
+    int64_t hnd2 = reg.AllocateCritical(SharedPointer<HandleRegistryTestEntry>(entry2));
 
     BOOST_REQUIRE(reg.Get(hnd0).Get() == entry0);
     BOOST_REQUIRE(!probe0.deleted);
@@ -109,7 +109,7 @@ BOOST_AUTO_TEST_CASE(TestCritical)
     HandleRegistryTestProbe closedProbe;
     HandleRegistryTestEntry* closedEntry = new HandleRegistryTestEntry(&closedProbe);
 
-    int64_t closedHnd = closedReg.AllocateCritical(SharedPointer<HandleRegistryEntry>(closedEntry));
+    int64_t closedHnd = closedReg.AllocateCritical(SharedPointer<HandleRegistryTestEntry>(closedEntry));
     BOOST_REQUIRE(closedHnd == -1);
     BOOST_REQUIRE(closedProbe.deleted);
 }
@@ -126,9 +126,9 @@ BOOST_AUTO_TEST_CASE(TestNonCritical)
     HandleRegistryTestEntry* entry1 = new HandleRegistryTestEntry(&probe1);
     HandleRegistryTestEntry* entry2 = new HandleRegistryTestEntry(&probe2);
 
-    int64_t hnd0 = reg.AllocateCritical(SharedPointer<HandleRegistryEntry>(entry0));
-    int64_t hnd1 = reg.Allocate(SharedPointer<HandleRegistryEntry>(entry1));
-    int64_t hnd2 = reg.Allocate(SharedPointer<HandleRegistryEntry>(entry2));
+    int64_t hnd0 = reg.AllocateCritical(SharedPointer<HandleRegistryTestEntry>(entry0));
+    int64_t hnd1 = reg.Allocate(SharedPointer<HandleRegistryTestEntry>(entry1));
+    int64_t hnd2 = reg.Allocate(SharedPointer<HandleRegistryTestEntry>(entry2));
 
     BOOST_REQUIRE(reg.Get(hnd0).Get() == entry0);
     BOOST_REQUIRE(!probe0.deleted);
@@ -168,7 +168,7 @@ BOOST_AUTO_TEST_CASE(TestNonCritical)
     HandleRegistryTestProbe closedProbe;
     HandleRegistryTestEntry* closedEntry = new HandleRegistryTestEntry(&closedProbe);
 
-    int64_t closedHnd = closedReg.Allocate(SharedPointer<HandleRegistryEntry>(closedEntry));
+    int64_t closedHnd = closedReg.Allocate(SharedPointer<HandleRegistryTestEntry>(closedEntry));
     BOOST_REQUIRE(closedHnd == -1);
     BOOST_REQUIRE(closedProbe.deleted);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core-test/src/reference_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/reference_test.cpp b/modules/platforms/cpp/core-test/src/reference_test.cpp
new file mode 100644
index 0000000..f5c3e8d
--- /dev/null
+++ b/modules/platforms/cpp/core-test/src/reference_test.cpp
@@ -0,0 +1,412 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef _MSC_VER
+    #define BOOST_TEST_DYN_LINK
+#endif
+
+#include <memory>
+
+#include <boost/test/unit_test.hpp>
+#include <boost/smart_ptr.hpp>
+#include <boost/interprocess/smart_ptr/unique_ptr.hpp>
+
+#include <ignite/reference.h>
+
+using namespace ignite;
+using namespace boost::unit_test;
+
+class LivenessMarker
+{
+public:
+    LivenessMarker(bool& flag) :
+        flag(flag)
+    {
+        flag = true;
+    }
+
+    LivenessMarker(const LivenessMarker& other) :
+        flag(other.flag)
+    {
+        // No-op.
+    }
+
+    LivenessMarker& operator=(const LivenessMarker& other)
+    {
+        flag = other.flag;
+
+        return *this;
+    }
+
+    ~LivenessMarker()
+    {
+        flag = false;
+    }
+
+private:
+    bool& flag;
+};
+
+class InstanceCounter
+{
+public:
+    InstanceCounter(int& counter) :
+        counter(&counter)
+    {
+        ++(*this->counter);
+    }
+
+    InstanceCounter(const InstanceCounter& other) :
+        counter(other.counter)
+    {
+        ++(*counter);
+    }
+
+    InstanceCounter& operator=(const InstanceCounter& other)
+    {
+        counter = other.counter;
+
+        ++(*counter);
+
+        return *this;
+    }
+
+    ~InstanceCounter()
+    {
+        --(*counter);
+    }
+
+private:
+    int* counter;
+};
+
+
+void TestFunction(Reference<LivenessMarker> ptr)
+{
+    Reference<LivenessMarker> copy(ptr);
+    Reference<LivenessMarker> copy2(ptr);
+}
+
+struct C1
+{
+    int c1;
+};
+
+struct C2
+{
+    int c2;
+};
+
+struct C3 : C1, C2
+{
+    int c3;
+};
+
+void TestFunction1(Reference<C1> c1, int expected)
+{
+    BOOST_CHECK_EQUAL(c1.Get().c1, expected);
+}
+
+void TestFunction2(Reference<C2> c2, int expected)
+{
+    BOOST_CHECK_EQUAL(c2.Get().c2, expected);
+}
+
+void TestFunction3(Reference<C3> c3, int expected)
+{
+    BOOST_CHECK_EQUAL(c3.Get().c3, expected);
+}
+
+void TestFunctionConst1(ConstReference<C1> c1, int expected)
+{
+    BOOST_CHECK_EQUAL(c1.Get().c1, expected);
+}
+
+void TestFunctionConst2(ConstReference<C2> c2, int expected)
+{
+    BOOST_CHECK_EQUAL(c2.Get().c2, expected);
+}
+
+void TestFunctionConst3(ConstReference<C3> c3, int expected)
+{
+    BOOST_CHECK_EQUAL(c3.Get().c3, expected);
+}
+
+BOOST_AUTO_TEST_SUITE(ReferenceTestSuite)
+
+BOOST_AUTO_TEST_CASE(StdSharedPointerTestBefore)
+{
+    bool objAlive = false;
+
+    std::shared_ptr<LivenessMarker> shared = std::make_shared<LivenessMarker>(objAlive);
+
+    BOOST_CHECK(objAlive);
+
+    {
+        Reference<LivenessMarker> smart = MakeReferenceFromSmartPointer(shared);
+
+        BOOST_CHECK(objAlive);
+
+        shared.reset();
+
+        BOOST_CHECK(objAlive);
+    }
+
+    BOOST_CHECK(!objAlive);
+}
+
+BOOST_AUTO_TEST_CASE(StdSharedPointerTestAfter)
+{
+    bool objAlive = false;
+
+    std::shared_ptr<LivenessMarker> shared = std::make_shared<LivenessMarker>(objAlive);
+
+    BOOST_CHECK(objAlive);
+
+    {
+        Reference<LivenessMarker> smart = MakeReferenceFromSmartPointer(shared);
+
+        BOOST_CHECK(objAlive);
+    }
+
+    BOOST_CHECK(objAlive);
+
+    shared.reset();
+
+    BOOST_CHECK(!objAlive);
+}
+
+BOOST_AUTO_TEST_CASE(StdAutoPointerTest)
+{
+    bool objAlive = false;
+
+    std::auto_ptr<LivenessMarker> autop(new LivenessMarker(objAlive));
+
+    BOOST_CHECK(objAlive);
+
+    {
+        Reference<LivenessMarker> smart = MakeReferenceFromSmartPointer(autop);
+
+        BOOST_CHECK(objAlive);
+    }
+
+    BOOST_CHECK(!objAlive);
+}
+
+BOOST_AUTO_TEST_CASE(StdUniquePointerTest)
+{
+    bool objAlive = false;
+
+    std::unique_ptr<LivenessMarker> unique(new LivenessMarker(objAlive));
+
+    BOOST_CHECK(objAlive);
+
+    {
+        Reference<LivenessMarker> smart = MakeReferenceFromSmartPointer(std::move(unique));
+
+        BOOST_CHECK(objAlive);
+    }
+
+    BOOST_CHECK(!objAlive);
+}
+
+BOOST_AUTO_TEST_CASE(BoostSharedPointerTestBefore)
+{
+    bool objAlive = false;
+
+    boost::shared_ptr<LivenessMarker> shared = boost::make_shared<LivenessMarker>(objAlive);
+
+    BOOST_CHECK(objAlive);
+
+    {
+        Reference<LivenessMarker> smart = MakeReferenceFromSmartPointer(shared);
+
+        BOOST_CHECK(objAlive);
+
+        shared.reset();
+
+        BOOST_CHECK(objAlive);
+    }
+
+    BOOST_CHECK(!objAlive);
+}
+
+BOOST_AUTO_TEST_CASE(BoostSharedPointerTestAfter)
+{
+    bool objAlive = false;
+
+    boost::shared_ptr<LivenessMarker> shared = boost::make_shared<LivenessMarker>(objAlive);
+
+    BOOST_CHECK(objAlive);
+
+    {
+        Reference<LivenessMarker> smart = MakeReferenceFromSmartPointer(shared);
+
+        BOOST_CHECK(objAlive);
+    }
+
+    BOOST_CHECK(objAlive);
+
+    shared.reset();
+
+    BOOST_CHECK(!objAlive);
+}
+
+BOOST_AUTO_TEST_CASE(PassingToFunction)
+{
+    bool objAlive = false;
+
+    std::shared_ptr<LivenessMarker> stdShared = std::make_shared<LivenessMarker>(objAlive);
+    std::unique_ptr<LivenessMarker> stdUnique(new LivenessMarker(objAlive));
+    std::auto_ptr<LivenessMarker> stdAuto(new LivenessMarker(objAlive));
+
+    boost::shared_ptr<LivenessMarker> boostShared = boost::make_shared<LivenessMarker>(objAlive);
+
+    TestFunction(MakeReferenceFromSmartPointer(stdShared));
+    TestFunction(MakeReferenceFromSmartPointer(std::move(stdUnique)));
+    TestFunction(MakeReferenceFromSmartPointer(stdAuto));
+
+    TestFunction(MakeReferenceFromSmartPointer(boostShared));
+}
+
+BOOST_AUTO_TEST_CASE(CopyTest)
+{
+    int instances = 0;
+
+    {
+        InstanceCounter counter(instances);
+
+        BOOST_CHECK_EQUAL(instances, 1);
+
+        {
+            Reference<InstanceCounter> copy = MakeReferenceFromCopy(counter);
+
+            BOOST_CHECK_EQUAL(instances, 2);
+        }
+
+        BOOST_CHECK_EQUAL(instances, 1);
+    }
+
+    BOOST_CHECK_EQUAL(instances, 0);
+}
+
+BOOST_AUTO_TEST_CASE(OwningPointerTest)
+{
+    int instances = 0;
+
+    {
+        InstanceCounter *counter = new InstanceCounter(instances);
+
+        BOOST_CHECK_EQUAL(instances, 1);
+
+        {
+            Reference<InstanceCounter> owned = MakeReferenceFromOwningPointer(counter);
+
+            BOOST_CHECK_EQUAL(instances, 1);
+        }
+
+        BOOST_CHECK_EQUAL(instances, 0);
+    }
+
+    BOOST_CHECK_EQUAL(instances, 0);
+}
+
+BOOST_AUTO_TEST_CASE(NonOwningPointerTest1)
+{
+    int instances = 0;
+
+    {
+        InstanceCounter counter(instances);
+
+        BOOST_CHECK_EQUAL(instances, 1);
+
+        {
+            Reference<InstanceCounter> copy = MakeReference(counter);
+
+            BOOST_CHECK_EQUAL(instances, 1);
+        }
+
+        BOOST_CHECK_EQUAL(instances, 1);
+    }
+
+    BOOST_CHECK_EQUAL(instances, 0);
+}
+
+BOOST_AUTO_TEST_CASE(NonOwningPointerTest2)
+{
+    int instances = 0;
+
+    InstanceCounter* counter = new InstanceCounter(instances);
+
+    BOOST_CHECK_EQUAL(instances, 1);
+
+    {
+        Reference<InstanceCounter> copy = MakeReference(*counter);
+
+        BOOST_CHECK_EQUAL(instances, 1);
+
+        delete counter;
+
+        BOOST_CHECK_EQUAL(instances, 0);
+    }
+
+    BOOST_CHECK_EQUAL(instances, 0);
+}
+
+BOOST_AUTO_TEST_CASE(CastTest)
+{
+    C3 testVal;
+
+    testVal.c1 = 1;
+    testVal.c2 = 2;
+    testVal.c3 = 3;
+
+    TestFunction1(MakeReference(testVal), 1);
+    TestFunction2(MakeReference(testVal), 2);
+    TestFunction3(MakeReference(testVal), 3);
+
+    TestFunction1(MakeReferenceFromCopy(testVal), 1);
+    TestFunction2(MakeReferenceFromCopy(testVal), 2);
+    TestFunction3(MakeReferenceFromCopy(testVal), 3);
+}
+
+BOOST_AUTO_TEST_CASE(ConstTest)
+{
+    C3 testVal;
+
+    testVal.c1 = 1;
+    testVal.c2 = 2;
+    testVal.c3 = 3;
+
+    TestFunctionConst1(MakeConstReference(testVal), 1);
+    TestFunctionConst2(MakeConstReference(testVal), 2);
+    TestFunctionConst3(MakeConstReference(testVal), 3);
+
+    TestFunctionConst1(MakeConstReferenceFromCopy(testVal), 1);
+    TestFunctionConst2(MakeConstReferenceFromCopy(testVal), 2);
+    TestFunctionConst3(MakeConstReferenceFromCopy(testVal), 3);
+
+    TestFunctionConst1(MakeReference(testVal), 1);
+    TestFunctionConst2(MakeReference(testVal), 2);
+    TestFunctionConst3(MakeReference(testVal), 3);
+
+    TestFunctionConst1(MakeReferenceFromCopy(testVal), 1);
+    TestFunctionConst2(MakeReferenceFromCopy(testVal), 2);
+    TestFunctionConst3(MakeReferenceFromCopy(testVal), 3);
+}
+
+BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/598b464f/modules/platforms/cpp/core/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/Makefile.am b/modules/platforms/cpp/core/Makefile.am
index 97523cf..758e68d 100644
--- a/modules/platforms/cpp/core/Makefile.am
+++ b/modules/platforms/cpp/core/Makefile.am
@@ -59,6 +59,7 @@ libignite_la_SOURCES = \
     src/impl/ignite_environment.cpp \
     src/impl/binary/binary_type_updater_impl.cpp \
     src/impl/handle_registry.cpp \
+    src/impl/cache/query/continuous/continuous_query_handle_impl.cpp \
     src/impl/cache/query/query_impl.cpp \
     src/impl/cache/cache_impl.cpp \
     src/impl/cache/query/query_batch.cpp \


[23/50] [abbrv] ignite git commit: IGNITE-4439: Added missing SUID.

Posted by yz...@apache.org.
IGNITE-4439: Added missing SUID.


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 1e2469fa80aeb554b12ff5e7c3465260f900dfc4
Parents: e2ac666
Author: devozerov <vo...@gridgain.com>
Authored: Fri Dec 23 12:25:21 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Dec 23 12:25:21 2016 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/util/AttributeNodeFilter.java | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1e2469fa/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java b/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java
index e2b972b..fed0d43 100644
--- a/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java
@@ -65,6 +65,9 @@ import org.jetbrains.annotations.Nullable;
  * {@code cpu-group} and {@code memory-group} attributes set to value {@code high}.
  */
 public class AttributeNodeFilter implements IgnitePredicate<ClusterNode> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
     /** Attributes. */
     private final Map<String, Object> attrs;
 


[19/50] [abbrv] ignite git commit: Merge ignite-1.8.2.

Posted by yz...@apache.org.
Merge ignite-1.8.2.


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 1b2afbb8d4032d23ee9a78987aec90a42e886700
Parents: 41dddb8 8372e69
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 22 16:51:00 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 22 16:51:00 2016 +0300

----------------------------------------------------------------------
 .../store/cassandra/CassandraCacheStore.java    |   6 +
 .../store/cassandra/datasource/DataSource.java  |   9 +
 .../java/org/apache/ignite/IgniteServices.java  |  16 +
 .../ignite/cache/store/CacheStoreAdapter.java   |   6 +
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |  19 +-
 .../store/jdbc/JdbcTypesDefaultTransformer.java | 112 ++--
 .../org/apache/ignite/internal/GridTopic.java   |   8 +-
 .../ignite/internal/IgniteServicesImpl.java     |   9 +-
 .../internal/binary/BinaryClassDescriptor.java  |  12 +-
 .../ignite/internal/binary/BinaryUtils.java     |  10 +-
 .../binary/builder/BinaryObjectBuilderImpl.java |  11 +-
 .../affinity/GridAffinityProcessor.java         |   2 +-
 .../cache/CacheStoreBalancingWrapper.java       |   6 +
 .../processors/cache/GridCacheAdapter.java      |   8 +-
 .../cache/GridCacheLoaderWriterStore.java       |   6 +
 .../GridCachePartitionExchangeManager.java      |  23 +-
 .../CacheDataStructuresManager.java             |   6 +-
 .../cache/query/GridCacheQueryManager.java      |   4 +-
 .../closure/GridClosureProcessor.java           |  31 +-
 .../internal/processors/job/GridJobWorker.java  |  76 ++-
 .../processors/odbc/OdbcRequestHandler.java     |  14 +-
 .../platform/PlatformContextImpl.java           |   2 +-
 .../dotnet/PlatformDotNetCacheStore.java        |  11 +
 .../platform/services/PlatformServices.java     |   2 +-
 .../platform/utils/PlatformUtils.java           |  28 +
 .../service/GridServiceProcessor.java           |  11 +-
 .../processors/service/GridServiceProxy.java    |  18 +-
 .../processors/task/GridTaskWorker.java         |   7 +
 .../communication/tcp/TcpCommunicationSpi.java  |   4 +-
 .../jdbc/JdbcTypesDefaultTransformerTest.java   | 283 +++++++++
 .../IgniteComputeTopologyExceptionTest.java     |   5 +-
 .../binary/BinaryMarshallerSelfTest.java        |  66 ++
 ...heapCacheMetricsForClusterGroupSelfTest.java | 141 +++++
 .../CacheOffHeapAndSwapMetricsSelfTest.java     | 621 -------------------
 ...LocalCacheOffHeapAndSwapMetricsSelfTest.java | 621 +++++++++++++++++++
 .../closure/GridClosureSerializationTest.java   | 177 ++++++
 ...gniteServiceProxyTimeoutInitializedTest.java | 284 +++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 .../IgniteCacheMetricsSelfTestSuite.java        |   6 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |   2 +
 .../processors/query/h2/IgniteH2Indexing.java   |  75 +--
 ...niteCachePartitionedFieldsQuerySelfTest.java |  25 +
 43 files changed, 2008 insertions(+), 779 deletions(-)
----------------------------------------------------------------------


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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/1b2afbb8/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
index ea3ae8b,c6b41d2..2e0fd10
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
@@@ -198,13 -198,19 +198,19 @@@ public class OdbcRequestHandler 
              qry.setDistributedJoins(distributedJoins);
              qry.setEnforceJoinOrder(enforceJoinOrder);
  
-             IgniteCache<Object, Object> cache = ctx.grid().cache(req.cacheName());
+             IgniteCache<Object, Object> cache0 = ctx.grid().cache(req.cacheName());
+ 
+             if (cache0 == null)
+                 return new OdbcResponse(OdbcResponse.STATUS_FAILED,
 -                        "Cache doesn't exist (did you configure it?): " + req.cacheName());
++                    "Cache doesn't exist (did you configure it?): " + req.cacheName());
+ 
+             IgniteCache<Object, Object> cache = cache0.withKeepBinary();
  
              if (cache == null)
-                 return new OdbcResponse(OdbcResponse.STATUS_FAILED, "Cache doesn't exist (did you configure it?): " +
-                     req.cacheName());
+                 return new OdbcResponse(OdbcResponse.STATUS_FAILED,
+                     "Can not get cache with keep binary: " + req.cacheName());
  
-             QueryCursor qryCur = cache.withKeepBinary().query(qry);
+             QueryCursor qryCur = cache.query(qry);
  
              qryCursors.put(qryId, new IgniteBiTuple<QueryCursor, Iterator>(qryCur, null));
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/1b2afbb8/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/1b2afbb8/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
index 22a7fa2,c266986..37727f5
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
@@@ -341,10 -340,10 +341,10 @@@ public class PlatformServices extends P
  
                  Object proxy = PlatformService.class.isAssignableFrom(d.serviceClass())
                      ? services.serviceProxy(name, PlatformService.class, sticky)
-                     : new GridServiceProxy<>(services.clusterGroup(), name, Service.class, sticky,
+                     : new GridServiceProxy<>(services.clusterGroup(), name, Service.class, sticky, 0,
                          platformCtx.kernalContext());
  
 -                return new ServiceProxyHolder(proxy, d.serviceClass());
 +                return new ServiceProxyHolder(proxy, d.serviceClass(), platformContext());
              }
          }
          return super.processInStreamOutObject(type, reader);

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


[35/50] [abbrv] ignite git commit: Fixed RAT.

Posted by yz...@apache.org.
Fixed RAT.


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

Branch: refs/heads/ignite-comm-balance-master
Commit: dbc8a0f832e7cebf362dd4818947df65075126ee
Parents: 8e7c852
Author: Andrey Novikov <an...@gridgain.com>
Authored: Fri Dec 23 16:34:21 2016 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Fri Dec 23 16:34:21 2016 +0700

----------------------------------------------------------------------
 modules/platforms/dotnet/build.bat | 11 ++++++++---
 1 file changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dbc8a0f8/modules/platforms/dotnet/build.bat
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/build.bat b/modules/platforms/dotnet/build.bat
index ef0a96f..5e70c27 100644
--- a/modules/platforms/dotnet/build.bat
+++ b/modules/platforms/dotnet/build.bat
@@ -1,6 +1,10 @@
-:: Licensed 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
+::
+:: 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
 ::
@@ -9,6 +13,7 @@
 :: 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.
+::
 
 :: Apache Ignite.NET build script runner, see build.ps1 for more details.
 powershell -executionpolicy remotesigned -file build.ps1 %*
\ No newline at end of file


[39/50] [abbrv] ignite git commit: IGNITE-4415 .NET: Fix duplicate entity set handling in EntityFramework caching

Posted by yz...@apache.org.
IGNITE-4415 .NET: Fix duplicate entity set handling in EntityFramework caching


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 781e5b7c93a1ee47299fc94519efbd0bea020940
Parents: 6ca8670
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Dec 23 16:53:59 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Dec 23 16:54:35 2016 +0300

----------------------------------------------------------------------
 .../EntityFrameworkCacheTest.cs                 | 54 +++++++++++++++++++-
 .../Impl/DbCommandInfo.cs                       | 21 +++++++-
 2 files changed, 72 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/781e5b7c/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 0e095f4..158df12 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheTest.cs
@@ -321,6 +321,55 @@ namespace Apache.Ignite.EntityFramework.Tests
         }
 
         /// <summary>
+        /// Queries with entity sets used multiple times are handled correctly.
+        /// </summary>
+        [Test]
+        public void TestDuplicateEntitySets()
+        {
+            using (var ctx = GetDbContext())
+            {
+                var blog = new Blog
+                {
+                    Name = "Foo",
+                    Posts = new List<Post>
+                    {
+                        new Post {Title = "Foo"},
+                        new Post {Title = "Foo"},
+                        new Post {Title = "Foo"},
+                        new Post {Title = "Bar"}
+                    }
+                };
+                ctx.Blogs.Add(blog);
+
+                Assert.AreEqual(5, ctx.SaveChanges());
+
+                var res = ctx.Blogs.Select(b => new
+                {
+                    X = b.Posts.FirstOrDefault(p => p.Title == b.Name),
+                    Y = b.Posts.Count(p => p.Title == b.Name)
+                }).ToArray();
+
+                Assert.AreEqual(1, res.Length);
+                Assert.AreEqual("Foo", res[0].X.Title);
+                Assert.AreEqual(3, res[0].Y);
+
+                // Modify and check updated result.
+                ctx.Posts.Remove(ctx.Posts.First(x => x.Title == "Foo"));
+                Assert.AreEqual(1, ctx.SaveChanges());
+
+                res = ctx.Blogs.Select(b => new
+                {
+                    X = b.Posts.FirstOrDefault(p => p.Title == b.Name),
+                    Y = b.Posts.Count(p => p.Title == b.Name)
+                }).ToArray();
+
+                Assert.AreEqual(1, res.Length);
+                Assert.AreEqual("Foo", res[0].X.Title);
+                Assert.AreEqual(2, res[0].Y);
+            }
+        }
+
+        /// <summary>
         /// Tests transactions created with BeginTransaction.
         /// </summary>
         [Test]
@@ -664,8 +713,11 @@ namespace Apache.Ignite.EntityFramework.Tests
         {
             TestUtils.RunMultiThreaded(CreateRemoveBlog, 4, 5);
 
+            // Run once again to force cleanup.
+            CreateRemoveBlog();
+
             // Wait for the cleanup to complete.
-            Thread.Sleep(2000);
+            Thread.Sleep(1000);
 
             // Only one version of data is in the cache.
             Assert.AreEqual(1, _cache.GetSize());

http://git-wip-us.apache.org/repos/asf/ignite/blob/781e5b7c/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs
index 7f18170..6b5db05 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs
@@ -127,6 +127,7 @@ namespace Apache.Ignite.EntityFramework.Impl
 
             expression.Accept(visitor);
 
+            // Should be sorted and unique.
             return visitor.EntitySets.ToArray();
         }
 
@@ -135,8 +136,9 @@ namespace Apache.Ignite.EntityFramework.Impl
         /// </summary>
         private class ScanExpressionVisitor : BasicCommandTreeVisitor
         {
-            /** */
-            private readonly List<EntitySetBase> _entitySets = new List<EntitySetBase>();
+            /** Unique and sorted entity sets. */
+            private readonly SortedSet<EntitySetBase> _entitySets = 
+                new SortedSet<EntitySetBase>(EntitySetComparer.Instance);
 
             /// <summary>
             /// Gets the entity sets.
@@ -154,5 +156,20 @@ namespace Apache.Ignite.EntityFramework.Impl
                 base.Visit(expression);
             }
         }
+
+        /// <summary>
+        /// Compares entity sets by name.
+        /// </summary>
+        private class EntitySetComparer : IComparer<EntitySetBase>
+        {
+            /** Default instance. */
+            public static readonly EntitySetComparer Instance = new EntitySetComparer();
+
+            /** <inheritdoc /> */
+            public int Compare(EntitySetBase x, EntitySetBase y)
+            {
+                return string.CompareOrdinal(x.Name, y.Name);
+            }
+        }
     }
 }


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

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

Branch: refs/heads/ignite-comm-balance-master
Commit: e2ac66678fa28c6744c4806bda87ffc6800018ec
Parents: 2591c16 848bb66
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Thu Dec 22 13:06:01 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Thu Dec 22 13:06:01 2016 -0800

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 bin/ignite.sh                                   |   9 +
 examples/pom-standalone.xml                     |   6 +
 modules/cassandra/pom.xml                       |  24 +-
 .../store/cassandra/CassandraCacheStore.java    |   6 +
 .../store/cassandra/datasource/DataSource.java  |   9 +
 .../java/org/apache/ignite/IgniteServices.java  |  16 +
 .../ignite/cache/store/CacheStoreAdapter.java   |   6 +
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |  19 +-
 .../store/jdbc/JdbcTypesDefaultTransformer.java | 112 +--
 .../org/apache/ignite/internal/GridTopic.java   |   8 +-
 .../ignite/internal/IgniteServicesImpl.java     |   9 +-
 .../internal/binary/BinaryClassDescriptor.java  |  12 +-
 .../ignite/internal/binary/BinaryUtils.java     |  10 +-
 .../binary/builder/BinaryObjectBuilderImpl.java |  11 +-
 .../affinity/GridAffinityProcessor.java         |   2 +-
 .../cache/CacheStoreBalancingWrapper.java       |   6 +
 .../processors/cache/GridCacheAdapter.java      | 734 +++++++------------
 .../cache/GridCacheLoaderWriterStore.java       |   6 +
 .../GridCachePartitionExchangeManager.java      |  60 +-
 .../processors/cache/GridCacheProcessor.java    |   2 +-
 .../processors/cache/GridCacheUtils.java        |  10 +-
 .../processors/cache/IgniteCacheProxy.java      |   8 -
 .../CacheDataStructuresManager.java             |   6 +-
 .../distributed/GridCacheTxRecoveryFuture.java  |   4 +-
 .../GridDistributedTxFinishResponse.java        |   6 -
 .../GridDistributedTxRemoteAdapter.java         |   2 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |   2 +-
 .../distributed/dht/GridDhtLockFuture.java      |   4 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |  74 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   | 125 ++--
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   7 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   4 +-
 .../dht/atomic/GridDhtAtomicCache.java          | 472 +++++++-----
 .../dht/colocated/GridDhtColocatedCache.java    |  13 -
 .../colocated/GridDhtColocatedLockFuture.java   |   4 +-
 .../GridDhtPartitionsExchangeFuture.java        |   6 +-
 .../distributed/near/GridNearAtomicCache.java   |  65 +-
 .../distributed/near/GridNearLockFuture.java    |   4 +-
 ...arOptimisticSerializableTxPrepareFuture.java |   4 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |  15 +-
 .../GridNearPessimisticTxPrepareFuture.java     |   4 +-
 .../near/GridNearTxFinishFuture.java            |  46 +-
 .../cache/distributed/near/GridNearTxLocal.java |  48 +-
 .../local/atomic/GridLocalAtomicCache.java      | 177 +----
 .../cache/query/GridCacheQueryManager.java      |   4 +-
 .../cache/store/CacheStoreManager.java          |   4 +-
 .../store/GridCacheStoreManagerAdapter.java     |  34 +-
 .../cache/store/GridCacheWriteBehindStore.java  |   6 +-
 .../cache/transactions/IgniteTxHandler.java     |  13 +-
 .../transactions/IgniteTxLocalAdapter.java      |  30 +-
 .../cache/transactions/IgniteTxManager.java     |   7 +-
 .../closure/GridClosureProcessor.java           |  31 +-
 .../GridCacheAtomicSequenceImpl.java            |  12 +-
 .../processors/hadoop/HadoopClassLoader.java    |   1 +
 .../processors/hadoop/HadoopJobProperty.java    |  10 +-
 .../processors/hadoop/HadoopTaskContext.java    |   8 +
 .../io/PartiallyOffheapRawComparatorEx.java     |  33 +
 .../internal/processors/job/GridJobWorker.java  |  76 +-
 .../processors/odbc/OdbcRequestHandler.java     |  14 +-
 .../platform/PlatformContextImpl.java           |   2 +-
 .../dotnet/PlatformDotNetCacheStore.java        |  11 +
 .../platform/services/PlatformServices.java     |   2 +-
 .../platform/utils/PlatformUtils.java           |  28 +
 .../service/GridServiceProcessor.java           |  11 +-
 .../processors/service/GridServiceProxy.java    |  18 +-
 .../processors/task/GridTaskWorker.java         |   7 +
 .../util/future/GridCompoundFuture.java         |  56 +-
 .../communication/tcp/TcpCommunicationSpi.java  |   4 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   2 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  14 +-
 .../jdbc/JdbcTypesDefaultTransformerTest.java   | 283 +++++++
 .../IgniteComputeTopologyExceptionTest.java     |   5 +-
 .../binary/BinaryMarshallerSelfTest.java        |  66 ++
 .../GridCacheMissingCommitVersionSelfTest.java  |   6 +-
 .../IgniteTxStoreExceptionAbstractSelfTest.java |   1 +
 ...heapCacheMetricsForClusterGroupSelfTest.java | 141 ++++
 ...achePartitionedAtomicSequenceTxSelfTest.java | 169 +++++
 .../CacheLockReleaseNodeLeaveTest.java          | 135 ++++
 ...nabledMultiNodeLongTxTimeoutFullApiTest.java |   2 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |  37 +-
 .../CacheOffHeapAndSwapMetricsSelfTest.java     | 621 ----------------
 ...LocalCacheOffHeapAndSwapMetricsSelfTest.java | 621 ++++++++++++++++
 .../closure/GridClosureSerializationTest.java   | 177 +++++
 ...lockMessageSystemPoolStarvationSelfTest.java |  14 +-
 ...gniteServiceProxyTimeoutInitializedTest.java | 284 +++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 .../IgniteCacheDataStructuresSelfTestSuite.java |   2 +
 .../IgniteCacheMetricsSelfTestSuite.java        |   6 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |   2 +
 .../hadoop/io/PartiallyRawComparator.java       |  33 +
 .../org/apache/ignite/hadoop/io/RawMemory.java  |  86 +++
 .../hadoop/io/TextPartiallyRawComparator.java   | 115 +++
 .../apache/ignite/hadoop/io/package-info.java   |  22 +
 ...DelegatingPartiallyOffheapRawComparator.java |  54 ++
 .../hadoop/impl/v2/HadoopV2TaskContext.java     |  21 +
 .../processors/hadoop/io/OffheapRawMemory.java  | 131 ++++
 .../hadoop/shuffle/HadoopShuffleJob.java        |   2 +-
 .../shuffle/collections/HadoopMultimapBase.java |   5 +-
 .../shuffle/collections/HadoopSkipList.java     |  14 +-
 .../hadoop/impl/HadoopTeraSortTest.java         |   7 +
 .../collections/HadoopAbstractMapTest.java      |   6 +
 .../processors/query/h2/IgniteH2Indexing.java   |  75 +-
 .../cache/IgniteCacheLargeResultSelfTest.java   |  15 +-
 ...niteCachePartitionedFieldsQuerySelfTest.java |  25 +
 .../IgniteCacheQuerySelfTestSuite.java          | 130 +++-
 .../IgniteCacheQuerySelfTestSuite2.java         | 111 +--
 .../IgniteH2IndexingSpiTestSuite.java           |  16 +-
 .../include/ignite/binary/binary_raw_reader.h   |  21 +
 .../ignite/impl/binary/binary_reader_impl.h     |   9 +-
 .../src/impl/binary/binary_reader_impl.cpp      |  17 +
 .../platforms/cpp/common/include/Makefile.am    |   5 +-
 .../common/include/ignite/common/concurrent.h   |  90 ++-
 .../include/ignite/common/reference_impl.h      | 286 ++++++++
 .../cpp/common/include/ignite/reference.h       | 564 ++++++++++++++
 .../cpp/common/project/vs/common.vcxproj        |   2 +
 .../common/project/vs/common.vcxproj.filters    |   6 +
 modules/platforms/cpp/core-test/Makefile.am     |   7 +-
 .../core-test/config/cache-query-continuous.xml |  87 +++
 .../cpp/core-test/config/cache-test.xml         |   1 +
 .../platforms/cpp/core-test/config/invalid.xml  |  39 +
 .../cpp/core-test/project/vs/core-test.vcxproj  |  13 +-
 .../project/vs/core-test.vcxproj.filters        |  11 +-
 .../cpp/core-test/src/cache_query_test.cpp      |  76 +-
 .../cpp/core-test/src/continuous_query_test.cpp | 611 +++++++++++++++
 .../cpp/core-test/src/handle_registry_test.cpp  |  18 +-
 .../cpp/core-test/src/interop_test.cpp          |  17 +-
 .../cpp/core-test/src/reference_test.cpp        | 412 +++++++++++
 modules/platforms/cpp/core/Makefile.am          |   1 +
 .../cpp/core/include/ignite/cache/cache.h       | 104 ++-
 .../cpp/core/include/ignite/cache/cache_entry.h |  40 +-
 .../ignite/cache/event/cache_entry_event.h      | 139 ++++
 .../cache/event/cache_entry_event_listener.h    |  71 ++
 .../cache/query/continuous/continuous_query.h   | 239 ++++++
 .../query/continuous/continuous_query_handle.h  | 133 ++++
 .../core/include/ignite/impl/cache/cache_impl.h | 116 ++-
 .../continuous/continuous_query_handle_impl.h   | 101 +++
 .../query/continuous/continuous_query_impl.h    | 351 +++++++++
 .../core/include/ignite/impl/handle_registry.h  |  62 +-
 .../include/ignite/impl/ignite_environment.h    |  34 +-
 modules/platforms/cpp/core/namespaces.dox       |  74 +-
 .../platforms/cpp/core/project/vs/core.vcxproj  |   7 +
 .../cpp/core/project/vs/core.vcxproj.filters    |  30 +
 modules/platforms/cpp/core/src/ignition.cpp     |  15 +-
 .../cpp/core/src/impl/cache/cache_impl.cpp      |  31 +
 .../continuous/continuous_query_handle_impl.cpp |  96 +++
 .../cpp/core/src/impl/handle_registry.cpp       | 102 ++-
 .../cpp/core/src/impl/ignite_environment.cpp    |  98 ++-
 modules/platforms/cpp/examples/Makefile.am      |   1 +
 modules/platforms/cpp/examples/configure.ac     |   1 +
 .../continuous-query-example/Makefile.am        |  58 ++
 .../config/continuous-query-example.xml         |  52 ++
 .../project/vs/continuous-query-example.vcxproj | 110 +++
 .../vs/continuous-query-example.vcxproj.filters |  35 +
 .../src/continuous_query_example.cpp            | 142 ++++
 .../examples/include/ignite/examples/person.h   |   2 +-
 .../cpp/examples/project/vs/ignite-examples.sln |   6 +
 .../platforms/cpp/jni/include/ignite/jni/java.h |   2 +-
 modules/platforms/cpp/jni/src/java.cpp          |   4 +-
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |   4 +-
 modules/platforms/cpp/project/vs/ignite.slnrel  |   3 +
 .../platforms/cpp/project/vs/ignite_x86.slnrel  |   3 +
 .../Cache/Query/CacheLinqTest.cs                |  21 +-
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |   3 +-
 .../Impl/CacheQueryExpressionVisitor.cs         |  12 +-
 .../Apache.Ignite.Linq/Impl/MethodVisitor.cs    |  34 +-
 modules/platforms/dotnet/Apache.Ignite.sln      |   2 +
 modules/platforms/dotnet/DEVNOTES.txt           |  12 +-
 modules/platforms/dotnet/build.bat              |  14 +
 modules/platforms/dotnet/build.ps1              | 211 ++++++
 171 files changed, 8519 insertions(+), 2409 deletions(-)
----------------------------------------------------------------------


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


[09/50] [abbrv] ignite git commit: IGNITE-4359 .NET: Fix Java timezone setting in tests

Posted by yz...@apache.org.
IGNITE-4359 .NET: Fix Java timezone setting in tests


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 700529a6e461d00362666de9de58a92cf7bf8bbe
Parents: 3bae858
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Dec 20 16:01:22 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Dec 20 16:01:22 2016 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs  | 3 +--
 modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs    | 3 ++-
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/700529a6/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
index 46979ef..e74f09f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
@@ -123,8 +123,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             {
                 BinaryConfiguration = new BinaryConfiguration(typeof(Person),
                     typeof(Organization), typeof(Address), typeof(Role), typeof(RoleKey), typeof(Numerics)),
-                GridName = gridName,
-                JvmOptions = { "-Duser.timezone=UTC" }
+                GridName = gridName
             };
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/700529a6/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 7d483bb..ad27398 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
@@ -55,7 +55,8 @@ namespace Apache.Ignite.Core.Tests
                 "-Xms1g",
                 "-Xmx4g",
                 "-ea",
-                "-DIGNITE_QUIET=true"
+                "-DIGNITE_QUIET=true",
+                "-Duser.timezone=UTC"
             }
             : new List<string>
             {


[50/50] [abbrv] ignite git commit: merged from master

Posted by yz...@apache.org.
merged from master


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

Branch: refs/heads/ignite-comm-balance-master
Commit: b8eb27f6061320227cd5be846c79387bd46dbb4c
Parents: 0706f1a 864af7e
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Dec 28 17:03:44 2016 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Dec 28 17:03:44 2016 +0300

----------------------------------------------------------------------
 .gitignore                                      |    1 +
 assembly/release-base.xml                       |    5 +
 bin/ignite.sh                                   |    9 +
 config/dotnet/default-dotnet.xml                |   45 +
 examples/pom-standalone.xml                     |    6 +
 examples/pom.xml                                |    2 +-
 examples/schema-import/pom.xml                  |    2 +-
 modules/aop/pom.xml                             |    2 +-
 modules/apache-license-gen/pom.xml              |    2 +-
 modules/aws/pom.xml                             |    2 +-
 modules/benchmarks/pom.xml                      |    2 +-
 .../jmh/misc/JmhIncrementBenchmark.java         |   95 +
 modules/camel/pom.xml                           |    2 +-
 modules/cassandra/pom.xml                       |   26 +-
 modules/cassandra/serializers/pom.xml           |    4 +-
 modules/cassandra/store/pom.xml                 |    4 +-
 .../store/cassandra/CassandraCacheStore.java    |    6 +
 .../store/cassandra/datasource/DataSource.java  |    9 +
 modules/clients/pom.xml                         |   12 +-
 modules/cloud/pom.xml                           |    2 +-
 modules/codegen/pom.xml                         |    2 +-
 modules/core/pom.xml                            |    2 +-
 .../java/org/apache/ignite/IgniteLogger.java    |    6 +-
 .../java/org/apache/ignite/IgniteServices.java  |   16 +
 .../ignite/cache/store/CacheStoreAdapter.java   |    6 +
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |   19 +-
 .../store/jdbc/JdbcTypesDefaultTransformer.java |  112 +-
 .../org/apache/ignite/internal/GridTopic.java   |   10 +-
 .../ignite/internal/IgniteServicesImpl.java     |    9 +-
 .../internal/binary/BinaryClassDescriptor.java  |   12 +-
 .../ignite/internal/binary/BinaryUtils.java     |   10 +-
 .../binary/builder/BinaryObjectBuilderImpl.java |   11 +-
 .../managers/communication/GridIoManager.java   |    5 +
 .../communication/GridIoMessageFactory.java     |   12 +-
 .../communication/IgniteIoTestMessage.java      |    2 +-
 .../affinity/GridAffinityProcessor.java         |    2 +-
 .../cache/CacheStoreBalancingWrapper.java       |    6 +
 .../processors/cache/GridCacheAdapter.java      |  723 ++--
 .../processors/cache/GridCacheEntryEx.java      |    4 +-
 .../cache/GridCacheLoaderWriterStore.java       |    6 +
 .../processors/cache/GridCacheMapEntry.java     |   17 +-
 .../GridCachePartitionExchangeManager.java      |   60 +-
 .../processors/cache/GridCacheProcessor.java    |    2 +-
 .../processors/cache/GridCacheUtils.java        |   10 +-
 .../CacheDataStructuresManager.java             |    6 +-
 .../distributed/GridCacheTxRecoveryFuture.java  |    4 +-
 .../GridDistributedTxRemoteAdapter.java         |    2 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |    2 +-
 .../distributed/dht/GridDhtLockFuture.java      |   27 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |   74 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |  125 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |    7 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    4 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  474 ++-
 .../dht/colocated/GridDhtColocatedCache.java    |   13 -
 .../colocated/GridDhtColocatedLockFuture.java   |    4 +-
 .../GridDhtPartitionsExchangeFuture.java        |    6 +-
 .../distributed/near/GridNearAtomicCache.java   |   65 +-
 .../distributed/near/GridNearLockFuture.java    |    4 +-
 ...arOptimisticSerializableTxPrepareFuture.java |    4 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   15 +-
 .../GridNearPessimisticTxPrepareFuture.java     |    4 +-
 .../near/GridNearTxFinishFuture.java            |   46 +-
 .../cache/distributed/near/GridNearTxLocal.java |   48 +-
 .../local/atomic/GridLocalAtomicCache.java      |  177 +-
 .../cache/query/GridCacheQueryManager.java      |    4 +-
 .../cache/store/CacheStoreManager.java          |    4 +-
 .../store/GridCacheStoreManagerAdapter.java     |   34 +-
 .../cache/store/GridCacheWriteBehindStore.java  |    6 +-
 .../cache/transactions/IgniteTxHandler.java     |   13 +-
 .../transactions/IgniteTxLocalAdapter.java      |   32 +-
 .../cache/transactions/IgniteTxManager.java     |    7 +-
 .../closure/GridClosureProcessor.java           |   31 +-
 .../GridCacheAtomicSequenceImpl.java            |   12 +-
 .../processors/hadoop/HadoopClassLoader.java    |   11 +
 .../processors/hadoop/HadoopJobProperty.java    |   23 +-
 .../hadoop/HadoopMapperAwareTaskOutput.java     |   32 +
 .../processors/hadoop/HadoopTaskContext.java    |    8 +
 .../processors/hadoop/HadoopTaskInfo.java       |   43 +
 .../io/PartiallyOffheapRawComparatorEx.java     |   33 +
 .../shuffle/HadoopDirectShuffleMessage.java     |  243 ++
 .../internal/processors/job/GridJobWorker.java  |   76 +-
 .../processors/odbc/OdbcRequestHandler.java     |   16 +-
 .../internal/processors/odbc/OdbcTableMeta.java |   15 +-
 .../platform/PlatformContextImpl.java           |    2 +-
 .../cache/PlatformCacheEntryFilterImpl.java     |    4 +-
 .../cache/PlatformCacheEntryProcessorImpl.java  |   43 +-
 .../affinity/PlatformAffinityFunction.java      |   52 +-
 .../callback/PlatformCallbackGateway.java       |  266 +-
 .../platform/callback/PlatformCallbackOp.java   |  206 +
 .../callback/PlatformCallbackUtils.java         |  544 +--
 .../platform/compute/PlatformAbstractJob.java   |    2 +-
 .../platform/compute/PlatformAbstractTask.java  |   15 +-
 .../platform/compute/PlatformClosureJob.java    |   12 +-
 .../platform/compute/PlatformFullJob.java       |   15 +-
 .../platform/compute/PlatformFullTask.java      |   18 +-
 .../PlatformStreamReceiverImpl.java             |    3 +
 .../dotnet/PlatformDotNetCacheStore.java        |   17 +-
 .../services/PlatformAbstractService.java       |   25 +-
 .../platform/services/PlatformServices.java     |    2 +-
 .../platform/utils/PlatformFutureUtils.java     |    4 +-
 .../platform/utils/PlatformUtils.java           |   36 +-
 .../service/GridServiceProcessor.java           |   11 +-
 .../processors/service/GridServiceProxy.java    |   18 +-
 .../processors/task/GridTaskWorker.java         |    7 +
 .../util/future/GridCompoundFuture.java         |   56 +-
 .../visor/service/VisorCancelServiceTask.java   |   70 +
 .../visor/service/VisorServiceDescriptor.java   |  132 +
 .../visor/service/VisorServiceTask.java         |   75 +
 .../internal/visor/util/VisorTaskUtils.java     |   15 +-
 .../apache/ignite/logger/java/JavaLogger.java   |    4 +-
 .../communication/tcp/TcpCommunicationSpi.java  |    8 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |    2 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   14 +-
 .../apache/ignite/util/AttributeNodeFilter.java |  108 +
 .../resources/META-INF/classnames.properties    |   65 +-
 .../core/src/main/resources/ignite.properties   |    2 +-
 .../jdbc/JdbcTypesDefaultTransformerTest.java   |  283 ++
 .../IgniteComputeTopologyExceptionTest.java     |    5 +-
 .../binary/BinaryMarshallerSelfTest.java        |   66 +
 .../GridCacheMissingCommitVersionSelfTest.java  |    6 +-
 .../processors/cache/GridCacheTestEntryEx.java  |    3 +-
 .../IgniteTxStoreExceptionAbstractSelfTest.java |    1 +
 ...heapCacheMetricsForClusterGroupSelfTest.java |  141 +
 ...achePartitionedAtomicSequenceTxSelfTest.java |  169 +
 .../CacheLockReleaseNodeLeaveTest.java          |  135 +
 ...nabledMultiNodeLongTxTimeoutFullApiTest.java |    2 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |   37 +-
 ...eCacheExpiryPolicyWithStoreAbstractTest.java |   40 +
 .../CacheOffHeapAndSwapMetricsSelfTest.java     |  621 ---
 ...LocalCacheOffHeapAndSwapMetricsSelfTest.java |  621 +++
 .../closure/GridClosureSerializationTest.java   |  177 +
 ...lockMessageSystemPoolStarvationSelfTest.java |   14 +-
 ...gniteServiceProxyTimeoutInitializedTest.java |  284 ++
 .../junits/logger/GridTestLog4jLogger.java      |    4 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    5 +
 .../IgniteCacheDataStructuresSelfTestSuite.java |    2 +
 .../IgniteCacheMetricsSelfTestSuite.java        |    6 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |    6 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    2 +
 .../util/AttributeNodeFilterSelfTest.java       |  184 +
 modules/docker/1.8.0/Dockerfile                 |   46 +
 modules/docker/1.8.0/run.sh                     |   51 +
 modules/docker/Dockerfile                       |    6 +-
 modules/extdata/p2p/pom.xml                     |    2 +-
 .../extdata/uri/modules/uri-dependency/pom.xml  |    2 +-
 modules/extdata/uri/pom.xml                     |    2 +-
 modules/flink/pom.xml                           |    2 +-
 modules/flume/pom.xml                           |    2 +-
 modules/gce/pom.xml                             |    2 +-
 modules/geospatial/pom.xml                      |    2 +-
 modules/hadoop/pom.xml                          |    9 +-
 .../hadoop/io/PartiallyRawComparator.java       |   33 +
 .../org/apache/ignite/hadoop/io/RawMemory.java  |   86 +
 .../hadoop/io/TextPartiallyRawComparator.java   |  115 +
 .../apache/ignite/hadoop/io/package-info.java   |   22 +
 .../IgniteHadoopClientProtocolProvider.java     |   70 +-
 .../processors/hadoop/HadoopMapperUtils.java    |   56 +
 .../hadoop/impl/fs/HadoopFileSystemsUtils.java  |   11 +
 .../hadoop/impl/proto/HadoopClientProtocol.java |   55 +-
 .../hadoop/impl/v2/HadoopV2Context.java         |   11 +
 ...DelegatingPartiallyOffheapRawComparator.java |   54 +
 .../processors/hadoop/impl/v2/HadoopV2Job.java  |   22 +-
 .../impl/v2/HadoopV2JobResourceManager.java     |   25 +-
 .../hadoop/impl/v2/HadoopV2MapTask.java         |   10 +
 .../hadoop/impl/v2/HadoopV2TaskContext.java     |   21 +
 .../processors/hadoop/io/OffheapRawMemory.java  |  131 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |    4 +
 .../hadoop/mapreduce/MapReduceClient.java       |  147 +
 .../hadoop/shuffle/HadoopShuffle.java           |   23 +-
 .../hadoop/shuffle/HadoopShuffleJob.java        |  391 +-
 .../shuffle/HadoopShuffleRemoteState.java       |    5 +-
 .../shuffle/collections/HadoopMultimapBase.java |    5 +-
 .../shuffle/collections/HadoopSkipList.java     |   14 +-
 .../shuffle/direct/HadoopDirectDataInput.java   |  166 +
 .../shuffle/direct/HadoopDirectDataOutput.java  |  221 ++
 .../direct/HadoopDirectDataOutputContext.java   |  100 +
 .../direct/HadoopDirectDataOutputState.java     |   54 +
 .../child/HadoopChildProcessRunner.java         |    2 +-
 .../hadoop/impl/HadoopAbstractSelfTest.java     |   13 +-
 .../impl/HadoopAbstractWordCountTest.java       |    6 +-
 .../hadoop/impl/HadoopFileSystemsTest.java      |    9 +
 .../hadoop/impl/HadoopJobTrackerSelfTest.java   |    4 +-
 .../impl/HadoopMapReduceEmbeddedSelfTest.java   |   22 +-
 .../impl/HadoopTaskExecutionSelfTest.java       |    4 +-
 .../hadoop/impl/HadoopTeraSortTest.java         |  383 ++
 ...opClientProtocolMultipleServersSelfTest.java |   93 +-
 .../client/HadoopClientProtocolSelfTest.java    |  232 +-
 .../collections/HadoopAbstractMapTest.java      |    6 +
 .../collections/HadoopSkipListSelfTest.java     |   14 +-
 .../HadoopExternalTaskExecutionSelfTest.java    |    2 +
 .../testsuites/IgniteHadoopTestSuite.java       |    3 +
 modules/hibernate/pom.xml                       |    2 +-
 modules/indexing/pom.xml                        |    2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   75 +-
 .../cache/IgniteCacheLargeResultSelfTest.java   |   15 +-
 ...niteCachePartitionedFieldsQuerySelfTest.java |   25 +
 .../IgniteCacheQuerySelfTestSuite.java          |  130 +-
 .../IgniteCacheQuerySelfTestSuite2.java         |  111 +-
 .../IgniteH2IndexingSpiTestSuite.java           |   16 +-
 modules/jcl/pom.xml                             |    2 +-
 modules/jms11/pom.xml                           |    2 +-
 modules/jta/pom.xml                             |    2 +-
 modules/kafka/pom.xml                           |    2 +-
 modules/log4j/pom.xml                           |    2 +-
 .../apache/ignite/logger/log4j/Log4JLogger.java |    4 +-
 modules/log4j2/pom.xml                          |    2 +-
 modules/mesos/pom.xml                           |    2 +-
 modules/mqtt/pom.xml                            |    2 +-
 modules/osgi-karaf/pom.xml                      |    2 +-
 modules/osgi-paxlogging/pom.xml                 |    2 +-
 modules/osgi/pom.xml                            |    2 +-
 modules/platforms/cpp/binary/Makefile.am        |    2 +-
 .../include/ignite/binary/binary_raw_reader.h   |   21 +
 .../ignite/impl/binary/binary_reader_impl.h     |    9 +-
 .../ignite/impl/binary/binary_type_manager.h    |    2 +-
 .../src/impl/binary/binary_reader_impl.cpp      |   17 +
 .../src/impl/binary/binary_type_manager.cpp     |    6 +-
 .../binary/src/impl/interop/interop_memory.cpp  |    2 +-
 modules/platforms/cpp/common/Makefile.am        |    2 +-
 .../platforms/cpp/common/include/Makefile.am    |    5 +-
 .../common/include/ignite/common/concurrent.h   |   90 +-
 .../include/ignite/common/dynamic_size_array.h  |    2 +-
 .../include/ignite/common/reference_impl.h      |  286 ++
 .../cpp/common/include/ignite/reference.h       |  564 +++
 .../cpp/common/project/vs/common.vcxproj        |    2 +
 .../common/project/vs/common.vcxproj.filters    |    6 +
 modules/platforms/cpp/configure.ac              |    2 +-
 modules/platforms/cpp/configure.acrel           |    2 +-
 modules/platforms/cpp/core-test/Makefile.am     |    9 +-
 .../core-test/config/cache-query-continuous.xml |   87 +
 .../cpp/core-test/config/cache-test.xml         |    1 +
 .../platforms/cpp/core-test/config/invalid.xml  |   39 +
 .../cpp/core-test/project/vs/core-test.vcxproj  |   13 +-
 .../project/vs/core-test.vcxproj.filters        |   11 +-
 .../cpp/core-test/src/cache_query_test.cpp      |   84 +-
 .../cpp/core-test/src/continuous_query_test.cpp |  611 +++
 .../cpp/core-test/src/handle_registry_test.cpp  |   18 +-
 .../cpp/core-test/src/interop_test.cpp          |   17 +-
 .../cpp/core-test/src/reference_test.cpp        |  422 ++
 modules/platforms/cpp/core/Makefile.am          |    3 +-
 .../cpp/core/include/ignite/cache/cache.h       |  104 +-
 .../cpp/core/include/ignite/cache/cache_entry.h |   40 +-
 .../ignite/cache/event/cache_entry_event.h      |  139 +
 .../cache/event/cache_entry_event_listener.h    |   71 +
 .../cache/query/continuous/continuous_query.h   |  239 ++
 .../query/continuous/continuous_query_handle.h  |  133 +
 .../include/ignite/cache/query/query_cursor.h   |    4 +-
 .../core/include/ignite/impl/cache/cache_impl.h |  116 +-
 .../continuous/continuous_query_handle_impl.h   |  101 +
 .../query/continuous/continuous_query_impl.h    |  351 ++
 .../core/include/ignite/impl/handle_registry.h  |   62 +-
 .../include/ignite/impl/ignite_environment.h    |   34 +-
 .../cpp/core/include/ignite/impl/operations.h   |    4 +-
 modules/platforms/cpp/core/namespaces.dox       |   74 +-
 .../platforms/cpp/core/project/vs/core.vcxproj  |    7 +
 .../cpp/core/project/vs/core.vcxproj.filters    |   30 +
 modules/platforms/cpp/core/src/ignition.cpp     |   15 +-
 .../cpp/core/src/impl/cache/cache_impl.cpp      |   32 +
 .../continuous/continuous_query_handle_impl.cpp |   96 +
 .../cpp/core/src/impl/handle_registry.cpp       |  102 +-
 .../cpp/core/src/impl/ignite_environment.cpp    |  146 +-
 modules/platforms/cpp/examples/Makefile.am      |    1 +
 modules/platforms/cpp/examples/configure.ac     |    3 +-
 .../continuous-query-example/Makefile.am        |   58 +
 .../config/continuous-query-example.xml         |   52 +
 .../project/vs/continuous-query-example.vcxproj |  110 +
 .../vs/continuous-query-example.vcxproj.filters |   35 +
 .../src/continuous_query_example.cpp            |  142 +
 .../examples/include/ignite/examples/person.h   |    2 +-
 .../cpp/examples/odbc-example/Makefile.am       |    2 +-
 .../cpp/examples/project/vs/ignite-examples.sln |    6 +
 .../cpp/examples/putget-example/Makefile.am     |    2 +-
 .../cpp/examples/query-example/Makefile.am      |    2 +-
 modules/platforms/cpp/ignite/Makefile.am        |    2 +-
 modules/platforms/cpp/jni/Makefile.am           |    2 +-
 .../platforms/cpp/jni/include/ignite/jni/java.h |   87 +-
 modules/platforms/cpp/jni/project/vs/module.def |    4 +-
 modules/platforms/cpp/jni/src/java.cpp          |  345 +-
 modules/platforms/cpp/odbc-test/Makefile.am     |    2 +-
 .../cpp/odbc-test/config/queries-default.xml    |  146 +
 .../odbc-test/config/queries-test-noodbc.xml    |   84 +-
 .../cpp/odbc-test/config/queries-test.xml       |   85 +-
 .../platforms/cpp/odbc-test/include/Makefile.am |    1 +
 .../cpp/odbc-test/include/complex_type.h        |  122 +
 .../odbc-test/include/sql_test_suite_fixture.h  |    3 +
 .../platforms/cpp/odbc-test/include/test_type.h |   20 +-
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |    5 +-
 .../project/vs/odbc-test.vcxproj.filters        |    3 +
 .../platforms/cpp/odbc-test/src/column_test.cpp |  189 +
 .../cpp/odbc-test/src/queries_test.cpp          |  177 +-
 .../odbc-test/src/sql_test_suite_fixture.cpp    |   17 +
 .../cpp/odbc-test/src/sql_types_test.cpp        |  131 +
 .../cpp/odbc-test/src/utility_test.cpp          |   27 +-
 modules/platforms/cpp/odbc/Makefile.am          |    2 +-
 .../cpp/odbc/include/ignite/odbc/utility.h      |    9 +-
 .../cpp/odbc/install/ignite-odbc-amd64.wxs      |    2 +-
 .../cpp/odbc/install/ignite-odbc-x86.wxs        |    2 +-
 modules/platforms/cpp/odbc/src/column.cpp       |   65 +-
 .../cpp/odbc/src/config/connection_info.cpp     |    6 +-
 modules/platforms/cpp/odbc/src/connection.cpp   |    2 +
 modules/platforms/cpp/odbc/src/odbc.cpp         |    5 +-
 modules/platforms/cpp/odbc/src/type_traits.cpp  |    3 +
 modules/platforms/cpp/odbc/src/utility.cpp      |   25 +
 modules/platforms/cpp/project/vs/ignite.slnrel  |    3 +
 .../platforms/cpp/project/vs/ignite_x86.slnrel  |    3 +
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Apache.Ignite.Core.Tests.csproj             |    1 +
 .../Cache/CacheAbstractTest.cs                  |  625 ---
 .../Cache/CacheAbstractTransactionalTest.cs     |  556 +++
 .../Cache/CacheLocalAtomicTest.cs               |    5 -
 .../Cache/CacheLocalTest.cs                     |    6 +-
 .../CachePartitionedAtomicNearEnabledTest.cs    |    5 -
 .../Cache/CachePartitionedAtomicTest.cs         |    5 -
 .../Cache/CachePartitionedNearEnabledTest.cs    |    7 +-
 .../Cache/CachePartitionedTest.cs               |    7 +-
 .../Cache/CacheReplicatedAtomicTest.cs          |    5 -
 .../Cache/CacheReplicatedTest.cs                |    7 +-
 .../Cache/Query/CacheLinqTest.cs                |   72 +-
 .../Query/CacheQueriesCodeConfigurationTest.cs  |   17 +-
 .../Cache/Query/CacheQueriesTest.cs             |    8 +
 .../Examples/Example.cs                         |    5 +-
 .../Examples/ExamplesTest.cs                    |  107 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Apache.Ignite.Core.Tests/ReconnectTest.cs   |   26 +-
 .../Services/ServicesTest.cs                    |    3 +-
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |    3 +-
 .../Apache.Ignite.Core.csproj                   |    1 +
 .../dotnet/Apache.Ignite.Core/Binary/IBinary.cs |    3 +-
 .../Apache.Ignite.Core/Cache/Query/QueryBase.cs |   15 +-
 .../Impl/Binary/BinaryUtils.cs                  |   16 +
 .../Impl/Binary/Io/BinaryStreamBase.cs          |    4 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |   29 +-
 .../Impl/Compute/ComputeTaskHolder.cs           |   14 +-
 .../Impl/Unmanaged/UnmanagedCallbackHandlers.cs |   79 +-
 .../Impl/Unmanaged/UnmanagedCallbackOp.cs       |   86 +
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        | 1229 +++---
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../EntityFrameworkCacheTest.cs                 |   54 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Impl/DbCommandInfo.cs                       |   21 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Impl/CacheQueryExpressionVisitor.cs         |   12 +-
 .../Apache.Ignite.Linq/Impl/ExpressionWalker.cs |    8 +
 .../Apache.Ignite.Linq/Impl/MethodVisitor.cs    |   34 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 modules/platforms/dotnet/Apache.Ignite.sln      |    2 +
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |    6 +-
 modules/platforms/dotnet/DEVNOTES.txt           |   12 +-
 modules/platforms/dotnet/build.bat              |   19 +
 modules/platforms/dotnet/build.ps1              |  211 +
 .../examples/Apache.Ignite.Examples/App.config  |    2 +
 .../DataStructures/AtomicSequenceExample.cs     |   26 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 modules/rest-http/pom.xml                       |    2 +-
 modules/scalar-2.10/pom.xml                     |    2 +-
 modules/scalar/pom.xml                          |    2 +-
 modules/schedule/pom.xml                        |    2 +-
 modules/schema-import-db/pom.xml                |    2 +-
 modules/schema-import/pom.xml                   |    2 +-
 modules/slf4j/pom.xml                           |    2 +-
 modules/spark-2.10/pom.xml                      |    2 +-
 modules/spark/pom.xml                           |    2 +-
 modules/spring/pom.xml                          |    2 +-
 modules/ssh/pom.xml                             |    2 +-
 modules/storm/pom.xml                           |    2 +-
 modules/tools/pom.xml                           |    2 +-
 modules/twitter/pom.xml                         |    2 +-
 modules/urideploy/pom.xml                       |    2 +-
 .../spi/deployment/uri/UriDeploymentSpi.java    |    2 +-
 modules/visor-console-2.10/pom.xml              |    2 +-
 modules/visor-console/pom.xml                   |    2 +-
 modules/visor-plugins/pom.xml                   |    2 +-
 modules/web-console/backend/app/agent.js        |   15 +
 modules/web-console/backend/app/browser.js      |   13 +
 modules/web-console/backend/app/mongo.js        |   24 +-
 modules/web-console/backend/routes/demo.js      |   17 +-
 modules/web-console/backend/routes/profile.js   |    3 +-
 .../web-console/backend/services/notebooks.js   |   14 +-
 .../web-console/backend/services/sessions.js    |    6 +-
 modules/web-console/backend/services/spaces.js  |   15 +
 modules/web-console/frontend/app/app.js         |    5 -
 .../controllers/reset-password.controller.js    |   14 +-
 .../frontend/app/data/event-groups.json         |  169 +
 .../frontend/app/data/event-types.json          |  169 -
 .../frontend/app/data/pom-dependencies.json     |   12 +-
 .../ui-ace-docker/ui-ace-docker.controller.js   |    2 +-
 .../directives/ui-ace-docker/ui-ace-docker.jade |    2 +-
 .../ui-ace-pojos/ui-ace-pojos.controller.js     |   12 +-
 .../ui-ace-pom/ui-ace-pom.controller.js         |    4 +-
 .../helpers/jade/form/form-field-dropdown.jade  |    5 +-
 .../helpers/jade/form/form-field-number.jade    |    3 +-
 .../app/helpers/jade/form/form-field-text.jade  |   19 +-
 .../frontend/app/helpers/jade/mixins.jade       |   52 +-
 .../frontend/app/modules/Demo/Demo.module.js    |    6 +-
 .../configuration/EventGroups.provider.js       |   30 -
 .../modules/configuration/Version.service.js    |    6 +-
 .../configuration/configuration.module.js       |   63 +-
 .../generator/AbstractTransformer.js            |   17 +
 .../modules/configuration/generator/Beans.js    |    5 +
 .../generator/ConfigurationGenerator.js         | 2795 +++++++-------
 .../configuration/generator/Custom.service.js   |   23 +
 .../configuration/generator/Docker.service.js   |    4 +-
 .../generator/JavaTransformer.service.js        | 2318 +++++------
 .../configuration/generator/Maven.service.js    |  234 ++
 .../configuration/generator/Pom.service.js      |  233 --
 .../generator/Properties.service.js             |    2 +-
 .../configuration/generator/Readme.service.js   |    2 +-
 .../generator/SharpTransformer.service.js       |  437 ++-
 .../generator/SpringTransformer.service.js      |  497 ++-
 .../defaults/Cache.platform.service.js          |   56 +
 .../generator/defaults/Cache.service.js         |  131 +
 .../defaults/Cluster.platform.service.js        |   43 +
 .../generator/defaults/Cluster.service.js       |  289 ++
 .../generator/defaults/Event-groups.service.js  |   27 +
 .../generator/defaults/IGFS.service.js          |   64 +
 .../defaults/cache.platform.provider.js         |   60 -
 .../generator/defaults/cache.provider.js        |  137 -
 .../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 ----------
 .../frontend/app/modules/sql/Notebook.data.js   |   11 +-
 .../app/modules/sql/Notebook.service.js         |    2 +-
 .../app/modules/sql/scan-filter-input.jade      |   39 -
 .../modules/sql/scan-filter-input.service.js    |   51 -
 .../frontend/app/modules/sql/sql.controller.js  |  211 +-
 .../frontend/app/modules/sql/sql.module.js      |    2 -
 .../app/modules/states/configuration.state.js   |    2 +
 .../configuration/caches/node-filter.jade       |    2 +-
 .../states/configuration/caches/query.jade      |    3 +
 .../states/configuration/caches/store.jade      |    4 +-
 .../configuration/clusters/checkpoint.jade      |   11 +-
 .../configuration/clusters/checkpoint/fs.jade   |    8 +-
 .../configuration/clusters/checkpoint/jdbc.jade |    8 +-
 .../configuration/clusters/checkpoint/s3.jade   |   25 +-
 .../clusters/collision/custom.jade              |    2 +-
 .../clusters/collision/job-stealing.jade        |    2 +-
 .../configuration/clusters/deployment.jade      |  129 +-
 .../states/configuration/clusters/events.jade   |    4 +-
 .../states/configuration/clusters/failover.jade |    4 +-
 .../clusters/general/discovery/zookeeper.jade   |    2 +-
 .../discovery/zookeeper/retrypolicy/custom.jade |    2 +-
 .../configuration/clusters/load-balancing.jade  |   23 +-
 .../configuration/clusters/logger/custom.jade   |    2 +-
 .../states/configuration/clusters/ssl.jade      |    2 +-
 .../summary/summary-zipper.service.js           |   37 +
 .../configuration/summary/summary.controller.js |  103 +-
 .../configuration/summary/summary.worker.js     |  123 +
 .../frontend/app/modules/user/Auth.service.js   |   11 +-
 .../frontend/app/services/JavaTypes.service.js  |   13 +-
 .../frontend/app/services/Messages.service.js   |   17 +-
 .../frontend/controllers/admin-controller.js    |  211 +-
 .../frontend/controllers/caches-controller.js   |   22 +-
 .../frontend/controllers/clusters-controller.js |   42 +-
 .../frontend/controllers/domains-controller.js  |   32 +-
 .../frontend/controllers/igfs-controller.js     |   20 +-
 .../frontend/controllers/profile-controller.js  |    3 +-
 .../gulpfile.babel.js/webpack/common.js         |   17 +-
 .../webpack/environments/development.js         |   14 +-
 .../webpack/environments/production.js          |    3 +-
 .../webpack/plugins/progress.js                 |   82 -
 modules/web-console/frontend/package.json       |  178 +-
 .../frontend/public/images/cache.png            |  Bin 23700 -> 24791 bytes
 .../frontend/public/images/domains.png          |  Bin 23828 -> 22131 bytes
 .../web-console/frontend/public/images/igfs.png |  Bin 14307 -> 14139 bytes
 .../frontend/public/images/query-chart.png      |  Bin 16637 -> 17142 bytes
 .../frontend/public/images/query-metadata.png   |  Bin 32298 -> 39361 bytes
 .../frontend/public/images/query-table.png      |  Bin 29189 -> 28065 bytes
 .../frontend/public/images/summary.png          |  Bin 31997 -> 33650 bytes
 .../stylesheets/_font-awesome-custom.scss       |   23 +-
 .../frontend/public/stylesheets/form-field.scss |   37 +
 .../frontend/public/stylesheets/style.scss      |  111 +-
 .../frontend/test/unit/JavaTypes.test.js        |   17 +-
 .../frontend/test/unit/Version.test.js          |    8 +-
 .../views/configuration/domains-import.jade     |    5 +-
 .../frontend/views/configuration/summary.jade   |   25 +-
 .../frontend/views/settings/admin.jade          |   85 +-
 .../frontend/views/sql/notebook-new.jade        |    2 +-
 modules/web-console/frontend/views/sql/sql.jade |  235 +-
 .../frontend/views/templates/alert.jade         |    2 +-
 .../frontend/views/templates/select.jade        |    2 +-
 modules/web-console/pom.xml                     |    2 +-
 modules/web-console/web-agent/pom.xml           |    2 +-
 modules/web/ignite-appserver-test/pom.xml       |    2 +-
 modules/web/ignite-websphere-test/pom.xml       |    2 +-
 modules/web/pom.xml                             |    2 +-
 modules/yardstick/pom.xml                       |    2 +-
 .../yardstick/IgniteAbstractBenchmark.java      |   30 +
 .../yardstick/IgniteBenchmarkArguments.java     |   11 +
 .../ignite/yardstick/IgniteBenchmarkUtils.java  |   42 +-
 .../apache/ignite/yardstick/PreloadLogger.java  |  155 +
 .../IgniteCacheRandomOperationBenchmark.java    |   25 +-
 modules/yarn/pom.xml                            |    2 +-
 modules/zookeeper/pom.xml                       |    2 +-
 pom.xml                                         |    2 +-
 506 files changed, 20688 insertions(+), 17066 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b8eb27f6/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
index 24ddcd2,2962540..c2e0452
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
@@@ -769,4 -769,4 +769,4 @@@ public enum GridTopic 
              return S.toString(T8.class, this);
          }
      }
--}
++}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8eb27f6/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index e7b2769,7ef7bc0..9b50051
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@@ -35,7 -35,6 +35,9 @@@ import java.util.concurrent.TimeUnit
  import java.util.concurrent.atomic.AtomicBoolean;
  import java.util.concurrent.atomic.AtomicLong;
  import java.util.concurrent.atomic.AtomicReference;
++import java.util.concurrent.atomic.AtomicLong;
++import java.util.concurrent.atomic.AtomicReference;
 +import java.util.concurrent.locks.Lock;
  import java.util.concurrent.locks.ReadWriteLock;
  import java.util.concurrent.locks.ReentrantReadWriteLock;
  import org.apache.ignite.IgniteCheckedException;
@@@ -58,9 -57,9 +60,12 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.pool.PoolProcessor;
  import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
  import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashSet;
 +import org.apache.ignite.internal.util.StripedCompositeReadWriteLock;
 +import org.apache.ignite.internal.util.future.GridFinishedFuture;
 +import org.apache.ignite.internal.util.future.GridFutureAdapter;
+ import org.apache.ignite.internal.util.GridSpinReadWriteLock;
+ import org.apache.ignite.internal.util.future.GridFinishedFuture;
+ import org.apache.ignite.internal.util.future.GridFutureAdapter;
  import org.apache.ignite.internal.util.lang.GridTuple3;
  import org.apache.ignite.internal.util.tostring.GridToStringInclude;
  import org.apache.ignite.internal.util.typedef.F;

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

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/b8eb27f6/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 554bb3d,1e73e79..b816612
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@@ -43,6 -43,7 +43,11 @@@ import org.apache.ignite.internal.manag
  import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceTest;
  import org.apache.ignite.internal.managers.communication.IgniteIoTestMessagesTest;
  import org.apache.ignite.internal.managers.communication.IgniteVariousConnectionNumberTest;
+ import org.apache.ignite.cache.store.jdbc.JdbcTypesDefaultTransformerTest;
++import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceMultipleConnectionsTest;
++import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceTest;
++import org.apache.ignite.internal.managers.communication.IgniteIoTestMessagesTest;
++import org.apache.ignite.internal.managers.communication.IgniteVariousConnectionNumberTest;
  import org.apache.ignite.internal.processors.cache.CacheAffinityCallSelfTest;
  import org.apache.ignite.internal.processors.cache.CacheDeferredDeleteSanitySelfTest;
  import org.apache.ignite.internal.processors.cache.CacheEntryProcessorCopySelfTest;


[24/50] [abbrv] ignite git commit: Web console beta-7.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/views/sql/sql.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/sql/sql.jade b/modules/web-console/frontend/views/sql/sql.jade
index e3f6461..03015e8 100644
--- a/modules/web-console/frontend/views/sql/sql.jade
+++ b/modules/web-console/frontend/views/sql/sql.jade
@@ -14,6 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
+include /app/helpers/jade/mixins.jade
 include /app/directives/ui-grid-settings/ui-grid-settings.jade
 
 mixin btn-toolbar(btn, click, tip, focusId)
@@ -56,10 +57,15 @@ mixin notebook-rename
             .input-tip
                 input.form-control(ng-model='notebook.editName' required ignite-on-enter='renameNotebook(notebook.editName)' ignite-on-escape='notebook.edit = false;')
         h1.pull-right
-            a.dropdown-toggle(data-toggle='dropdown' bs-dropdown='scrollParagraphs' data-placement='bottom-right') Scroll to query
+            a.dropdown-toggle(style='margin-right: 20px' data-toggle='dropdown' bs-dropdown='scrollParagraphs' data-placement='bottom-right') Scroll to query
                 span.caret
-            .btn-group(style='margin-top: 2px')
-                +btn-toolbar('fa-plus', 'addParagraph()', 'Add new query')
+            button.btn.btn-default(style='margin-top: 2px' ng-click='addQuery()' ignite-on-click-focus=focusId)
+                i.fa.fa-fw.fa-plus
+                | Add query
+
+            button.btn.btn-default(style='margin-top: 2px' ng-click='addScan()' ignite-on-click-focus=focusId)
+                i.fa.fa-fw.fa-plus
+                | Add scan
 
 mixin notebook-error
     h2 Failed to load notebook
@@ -68,7 +74,7 @@ mixin notebook-error
 
 mixin paragraph-rename
     .col-sm-6(ng-hide='paragraph.edit')
-        i.tipLabel.fa(ng-class='paragraphExpanded(paragraph) ? "fa-chevron-circle-down" : "fa-chevron-circle-right"')
+        i.fa(ng-class='paragraphExpanded(paragraph) ? "fa-chevron-circle-down" : "fa-chevron-circle-right"')
         label {{paragraph.name}}
 
         .btn-group(ng-hide='notebook.paragraphs.length > 1')
@@ -85,51 +91,45 @@ mixin paragraph-rename
             input.form-control(id='paragraph-name-{{paragraph.id}}' ng-model='paragraph.editName' required ng-click='$event.stopPropagation();' ignite-on-enter='renameParagraph(paragraph, paragraph.editName)' ignite-on-escape='paragraph.edit = false')
 
 mixin query-settings
-    label.tipLabel Refresh rate:
-        button.btn.btn-default.fa.fa-clock-o.tipLabel(title='Click to show refresh rate dialog' ng-class='{"btn-info": paragraph.rate && paragraph.rate.installed}' bs-popover data-template-url='/sql/paragraph-rate.html' data-placement='left' data-auto-close='1' data-trigger='click') {{rateAsString(paragraph)}}
-    label.tipLabel Page size:
-        button.btn.btn-default.select-toggle.tipLabel(ng-model='paragraph.pageSize' bs-options='item for item in pageSizes' bs-select bs-tooltip data-placement='bottom-right' data-title='Max number of rows to show in query result as one page')
-    label.margin-left-dflt(title='Fetch first page of results only')
-        input(type='checkbox' ng-model='paragraph.firstPageOnly')
-        span Fetch first page only
-    label.margin-left-dflt(title='Execute query locally on selected node.\nNode selection dialog will be shown before query execution.')
-        input(type='checkbox' ng-model='paragraph.localQry')
-        span Local query
+    label.tipLabel(bs-tooltip data-placement='bottom' data-title='Configure periodical execution of last successfully executed query') Refresh rate:
+        button.btn.btn-default.fa.fa-clock-o.tipLabel(ng-class='{"btn-info": paragraph.rate && paragraph.rate.installed}' bs-popover data-template-url='/sql/paragraph-rate.html' data-placement='left' data-auto-close='1' data-trigger='click') {{rateAsString(paragraph)}}
+
+    label.tipLabel(bs-tooltip data-placement='bottom' data-title='Max number of rows to show in query result as one page') Page size:
+        button.btn.btn-default.select-toggle.tipLabel(ng-model='paragraph.pageSize' bs-select bs-options='item for item in pageSizes')
+
+    label.tipLabel(bs-tooltip data-placement='bottom' data-title='Limit query max results to specified number of pages') Max pages:
+        button.btn.btn-default.select-toggle.tipLabel(ng-model='paragraph.maxPages' bs-select bs-options='item.value as item.label for item in maxPages')
+
+    label.tipLabel(ng-if='nonCollocatedJoinsAvailable(paragraph)' bs-tooltip data-placement='bottom' data-title='Non-collocated joins is a special mode that allow to join data across cluster without collocation.<br/>\
+        Nested joins are not supported for now.<br/>\
+        <b>NOTE</b>: In some cases it may consume more heap memory or may take a long time than collocated joins.' data-trigger='hover')
+        input(type='checkbox' ng-model='paragraph.nonCollocatedJoins')
+        span Allow non-collocated joins
 
 mixin query-actions
-    .btn-group(bs-tooltip='' data-title='{{actionTooltip(paragraph, "execute", true)}}' data-placement='bottom')
-        button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, true)' ng-click='execute(paragraph)') Execute
-        button.btn.btn-primary.dropdown-toggle(
-            ng-disabled='!actionAvailable(paragraph, true)'
-            bs-dropdown=''
-            data-container='body'
-            data-placement='bottom-right'
-        )
-            span.caret
-        ul.dropdown-menu(role='menu')
-            li #[a(href='javascript:void(0)' ng-click='execute(paragraph)') Execute]
-            li #[a(href='javascript:void(0)' ng-if='nonCollocatedJoinsAvailable(paragraph)' ng-click='execute(paragraph, true)') Execute non collocated joins]
-    .btn-group(bs-tooltip='' data-title='{{actionTooltip(paragraph, "execute scan", false)}}' data-placement='bottom')
-        button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, false)' ng-click='scan(paragraph)') Scan
-        button.btn.btn-primary.dropdown-toggle(
-            ng-disabled='!actionAvailable(paragraph, false)'
-            bs-dropdown=''
-            data-container='body'
-            data-placement='bottom-right'
-        )
-            span.caret
-        ul.dropdown-menu(role='menu')
-            li #[a(href='javascript:void(0)' ng-click='scan(paragraph)') Scan]
-            li #[a(href='javascript:void(0)' ng-click='actionAvailable(paragraph, false) && scanWithFilter(paragraph)') Scan with filter]
+    button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, true)' ng-click='execute(paragraph)') Execute
+    button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, true)' ng-click='execute(paragraph, true)') Execute on selected node
+
     a.btn.btn-default(ng-disabled='!actionAvailable(paragraph, true)' ng-click='explain(paragraph)' data-placement='bottom' bs-tooltip='' data-title='{{actionTooltip(paragraph, "explain", true)}}') Explain
 
-mixin query-controls
-    .sql-controls
-        +query-actions()
-        .pull-right
-            +query-settings()
+mixin table-result-heading-query
+    .total.row
+        .col-xs-4
+            +ui-grid-settings
+            label Page: #[b {{paragraph.page}}]
+            label.margin-left-dflt Results so far: #[b {{paragraph.rows.length + paragraph.total}}]
+            label.margin-left-dflt Duration: #[b {{paragraph.duration | duration}}]
+        .col-xs-4
+            div(ng-if='paragraph.qryType === "query"')
+                +result-toolbar
+        .col-xs-4
+            .pull-right
+                .btn-group(ng-disabled='paragraph.loading')
+                    button.btn.btn-primary(ng-click='exportCsv(paragraph)' bs-tooltip data-title='{{actionTooltip(paragraph, "export", false)}}') Export
+                    button.btn.btn-primary.dropdown-toggle(id='export-item-dropdown' data-toggle='dropdown' data-container='body' bs-dropdown='exportDropdown' data-placement='bottom-right')
+                        span.caret
 
-mixin table-result
+mixin table-result-heading-scan
     .total.row
         .col-xs-4
             +ui-grid-settings
@@ -137,17 +137,16 @@ mixin table-result
             label.margin-left-dflt Results so far: #[b {{paragraph.rows.length + paragraph.total}}]
             label.margin-left-dflt Duration: #[b {{paragraph.duration | duration}}]
         .col-xs-4
-            +result-toolbar
+            div(ng-if='paragraph.qryType === "query"')
+                +result-toolbar
         .col-xs-4
             .pull-right
-                label(style='margin-right: 10px;')
-                    input(type='checkbox' ng-model='paragraph.systemColumns' ng-change='toggleSystemColumns(paragraph)' ng-disabled='paragraph.disabledSystemColumns')
-                    span Show _KEY, _VAL columns
                 .btn-group(ng-disabled='paragraph.loading')
                     button.btn.btn-primary(ng-click='exportCsv(paragraph)' bs-tooltip data-title='{{actionTooltip(paragraph, "export", false)}}') Export
                     button.btn.btn-primary.dropdown-toggle(id='export-item-dropdown' data-toggle='dropdown' data-container='body' bs-dropdown='exportDropdown' data-placement='bottom-right')
                         span.caret
-    
+
+mixin table-result-body
     .grid(ui-grid='paragraph.gridOptions' ui-grid-resize-columns ui-grid-exporter)
 
 mixin chart-result
@@ -166,12 +165,99 @@ mixin chart-result
                 +result-toolbar
         label.margin-top-dflt Charts do not support #[b Explain] and #[b Scan] query
 
+mixin paragraph-scan
+    .panel-heading(bs-collapse-toggle)
+        .row
+            +paragraph-rename
+    .panel-collapse(role='tabpanel' bs-collapse-target)
+        .col-sm-12.sql-controls
+            .col-sm-3
+                +dropdown-required('Cache:', 'paragraph.cacheName', '"cache"', 'true', 'false', 'Choose cache', 'caches')
+            .col-sm-3
+                +text-enabled('Filter:', 'paragraph.filter', '"filter"', true, false, 'Enter filter')
+                    label.btn.btn-default.ignite-form-field__btn(ng-click='paragraph.caseSensitive = !paragraph.caseSensitive')
+                        input(type='checkbox' ng-model='paragraph.caseSensitive')
+                        span(bs-tooltip data-title='Select this checkbox for case sensitive search') Cs
+            label.tipLabel(bs-tooltip data-placement='bottom' data-title='Max number of rows to show in query result as one page') Page size:
+                button.btn.btn-default.select-toggle.tipLabel(ng-model='paragraph.pageSize' bs-select bs-options='item for item in pageSizes')
+
+        .col-sm-12.sql-controls
+            button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, false)' ng-click='scan(paragraph)')
+                | Scan
+            button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, false)' ng-click='scan(paragraph, true)')
+                | Scan on selected node
+
+        .col-sm-12.sql-result(ng-if='paragraph.queryExecuted()' ng-switch='paragraph.resultType()')
+            .error(ng-switch-when='error') Error: {{paragraph.errMsg}}
+            .empty(ng-switch-when='empty') Result set is empty
+            .table(ng-switch-when='table')
+                +table-result-heading-scan
+                +table-result-body
+            .footer.clearfix()
+                .pull-left
+                    | Showing results for scan of #[b{{ paragraph.queryArgs.cacheName | defaultName }}]
+                    span(ng-if='paragraph.queryArgs.filter') &nbsp; with filter: #[b {{ paragraph.queryArgs.filter }}]
+                    span(ng-if='paragraph.queryArgs.localNid') &nbsp; on node: #[b {{ paragraph.queryArgs.localNid | limitTo:8 }}]
+
+                -var nextVisibleCondition = 'paragraph.resultType() != "error" && paragraph.queryId && paragraph.nonRefresh() && (paragraph.table() || paragraph.chart() && !paragraph.scanExplain())'
+
+                .pull-right(ng-show='#{nextVisibleCondition}' ng-class='{disabled: paragraph.loading}' ng-click='!paragraph.loading && nextPage(paragraph)')
+                    i.fa.fa-chevron-circle-right
+                    a Next
+
+mixin paragraph-query
+    .row.panel-heading(bs-collapse-toggle)
+        +paragraph-rename
+    .panel-collapse(role='tabpanel' bs-collapse-target)
+        .col-sm-12
+            .col-xs-8.col-sm-9(style='border-right: 1px solid #eee')
+                .sql-editor(ignite-ace='{onLoad: aceInit(paragraph), theme: "chrome", mode: "sql", require: ["ace/ext/language_tools"],' +
+                'advanced: {enableSnippets: false, enableBasicAutocompletion: true, enableLiveAutocompletion: true}}'
+                ng-model='paragraph.query')
+            .col-xs-4.col-sm-3
+                div(ng-show='caches.length > 0' style='padding: 5px 10px' st-table='displayedCaches' st-safe-src='caches')
+                    lable.labelField.labelFormField Caches:
+                    i.fa.fa-database.tipField(title='Click to show cache types metadata dialog' bs-popover data-template-url='/sql/cache-metadata.html' data-placement='bottom' data-trigger='click' data-container='#{{ paragraph.id }}')
+                    .input-tip
+                        input.form-control(type='text' st-search='label' placeholder='Filter caches...')
+                    table.links
+                        tbody.scrollable-y(style='max-height: 15em; display: block;')
+                            tr(ng-repeat='cache in displayedCaches track by cache.name')
+                                td(style='width: 100%')
+                                    input.labelField(id='cache_{{ [paragraph.id, $index].join("_") }}' type='radio' value='{{cache.name}}' ng-model='paragraph.cacheName')
+                                    label(for='cache_{{ [paragraph.id, $index].join("_") }} ' ng-bind-html='cache.label')
+                .empty-caches(ng-show='displayedCaches.length == 0 && caches.length != 0')
+                    label Wrong caches filter
+                .empty-caches(ng-show='caches.length == 0')
+                    label No caches
+        .col-sm-12.sql-controls
+            +query-actions
+
+            .pull-right
+                +query-settings
+        .col-sm-12.sql-result(ng-if='paragraph.queryExecuted()' ng-switch='paragraph.resultType()')
+            .error(ng-switch-when='error') Error: {{paragraph.errMsg}}
+            .empty(ng-switch-when='empty') Result set is empty
+            .table(ng-switch-when='table')
+                +table-result-heading-query
+                +table-result-body
+            .chart(ng-switch-when='chart')
+                +chart-result
+            .footer.clearfix
+                a.pull-left(ng-click='showResultQuery(paragraph)') Show query
+
+                -var nextVisibleCondition = 'paragraph.resultType() != "error" && paragraph.queryId && paragraph.nonRefresh() && (paragraph.table() || paragraph.chart() && !paragraph.scanExplain())'
+
+                .pull-right(ng-show='#{nextVisibleCondition}' ng-class='{disabled: paragraph.loading}' ng-click='!paragraph.loading && nextPage(paragraph)')
+                    i.fa.fa-chevron-circle-right
+                    a Next
+
 .row(ng-controller='sqlController')
     .docs-content
         .row(ng-if='notebook' bs-affix style='margin-bottom: 20px;')
             +notebook-rename
 
-        ignite-information(data-title='With SQL notebook you can' style='margin-top: 0; margin-bottom: 30px')
+        ignite-information(data-title='With query notebook you can' style='margin-top: 0; margin-bottom: 30px')
             ul
                 li Create any number of queries
                 li Execute and explain SQL queries
@@ -184,46 +270,9 @@ mixin chart-result
         div(ng-if='notebook' ignite-loading='sqlLoading' ignite-loading-text='{{ loadingText }}' ignite-loading-position='top')
             .docs-body.paragraphs
                 .panel-group(bs-collapse ng-model='notebook.expandedParagraphs' data-allow-multiple='true' data-start-collapsed='false')
-                    .panel.panel-default(ng-repeat='paragraph in notebook.paragraphs' id='{{paragraph.id}}')
-                        .panel-heading(bs-collapse-toggle)
-                            .row
-                                +paragraph-rename
-                        .panel-collapse(role='tabpanel' bs-collapse-target)
-                            .col-sm-12
-                                .col-xs-8.col-sm-9(style='border-right: 1px solid #eee')
-                                    .sql-editor(ignite-ace='{onLoad: aceInit(paragraph), theme: "chrome", mode: "sql", require: ["ace/ext/language_tools"],' +
-                                    'advanced: {enableSnippets: false, enableBasicAutocompletion: true, enableLiveAutocompletion: true}}'
-                                    ng-model='paragraph.query')
-                                .col-xs-4.col-sm-3
-                                    div(ng-show='caches.length > 0' style='padding: 5px 10px' st-table='displayedCaches' st-safe-src='caches')
-                                        lable.labelField.labelFormField Caches:
-                                        i.fa.fa-database.tipField(title='Click to show cache types metadata dialog' bs-popover data-template-url='/sql/cache-metadata.html' data-placement='bottom' data-trigger='click' data-container='#{{ paragraph.id }}')
-                                        .input-tip
-                                            input.form-control(type='text' st-search='label' placeholder='Filter caches...')
-                                        table.links
-                                            tbody.scrollable-y(style='max-height: 15em; display: block;')
-                                                tr(ng-repeat='cache in displayedCaches track by cache.name')
-                                                    td(style='width: 100%')
-                                                        input.labelField(id='cache_{{ [paragraph.id, $index].join("_") }}' type='radio' value='{{cache.name}}' ng-model='paragraph.cacheName')
-                                                        label(for='cache_{{ [paragraph.id, $index].join("_") }} ' ng-bind='cache.label')
-                                    .empty-caches(ng-show='displayedCaches.length == 0 && caches.length != 0')
-                                        label Wrong caches filter
-                                    .empty-caches(ng-show='caches.length == 0')
-                                        label No caches
-                            .col-sm-12
-                                +query-controls
-                            .col-sm-12.sql-result(ng-if='paragraph.queryExecuted()' ng-switch='paragraph.resultType()')
-                                .error(ng-switch-when='error') Error: {{paragraph.errMsg}}
-                                .empty(ng-switch-when='empty') Result set is empty
-                                .table(ng-switch-when='table')
-                                    +table-result
-                                .chart(ng-switch-when='chart')
-                                    +chart-result
-                                .footer.clearfix
-                                    a.pull-left(ng-click='showResultQuery(paragraph)') Show query
-
-                                    -var nextVisibleCondition = 'paragraph.resultType() != "error" && paragraph.queryId && paragraph.nonRefresh() && (paragraph.table() || paragraph.chart() && !paragraph.scanExplain())'
-
-                                    .pull-right(ng-show=nextVisibleCondition ng-class='{disabled: paragraph.loading}' ng-click='!paragraph.loading && nextPage(paragraph)')
-                                        i.fa.fa-chevron-circle-right
-                                        a Next
+
+                    .panel-paragraph(ng-repeat='paragraph in notebook.paragraphs' id='{{paragraph.id}}' ng-form='form_{{paragraph.id}}')
+                        .panel.panel-default(ng-if='paragraph.qryType === "scan"')
+                            +paragraph-scan
+                        .panel.panel-default(ng-if='paragraph.qryType === "query"')
+                            +paragraph-query

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/views/templates/alert.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/templates/alert.jade b/modules/web-console/frontend/views/templates/alert.jade
index 182ba99..d30d2fd 100644
--- a/modules/web-console/frontend/views/templates/alert.jade
+++ b/modules/web-console/frontend/views/templates/alert.jade
@@ -16,6 +16,6 @@
 
 .alert(ng-show='type' ng-class='[type ? "alert-" + type : null]')
     button.close(type='button', ng-if='dismissable', ng-click='$hide()') &times;
-    i.alert-icon.fa(ng-if='icon' ng-class='[icon]')
+    i.alert-icon(ng-if='icon' ng-class='[icon]')
     span.alert-title(ng-bind-html='title')
     span.alert-content(ng-bind-html='content')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/views/templates/select.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/templates/select.jade b/modules/web-console/frontend/views/templates/select.jade
index 5b6cc01..aa6a2ef 100644
--- a/modules/web-console/frontend/views/templates/select.jade
+++ b/modules/web-console/frontend/views/templates/select.jade
@@ -23,4 +23,4 @@ ul.select.dropdown-menu(tabindex='-1' ng-show='$isVisible()' role='select')
         hr(ng-if='match.value == undefined' style='margin: 5px 0')
         a(id='li-dropdown-item-{{$index}}'  role='menuitem' tabindex='-1' ng-class='{active: $isActive($index)}' ng-click='$select($index, $event)' bs-tooltip='widthIsSufficient && !widthIsSufficient("li-dropdown-item-{{$index}}", $index, match.label) ? match.label : ""' data-placement='right auto')
             i(class='{{$iconCheckmark}}' ng-if='$isActive($index)' ng-class='{active: $isActive($index)}')
-            span(ng-bind='match.label')
+            span(ng-bind-html='match.label')


[46/50] [abbrv] ignite git commit: IGNITE-4367 .NET: Fix flaky tests - add topology checks to ReconnectTest.TestClusterRestart

Posted by yz...@apache.org.
IGNITE-4367 .NET: Fix flaky tests - add topology checks to ReconnectTest.TestClusterRestart


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 8dd03222f87dc0b0b6b0f0a86adb5ec196592e38
Parents: 10b2b97
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Dec 27 20:11:37 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Dec 27 20:11:37 2016 +0300

----------------------------------------------------------------------
 .../platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs  | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8dd03222/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 253a10f..5a2d2d0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs
@@ -53,8 +53,13 @@ namespace Apache.Ignite.Core.Tests
             };
 
             var server = Ignition.Start(serverCfg);
+
+            Assert.AreEqual(1, server.GetCluster().GetNodes().Count);
+
             var client = Ignition.Start(clientCfg);
 
+            Assert.AreEqual(2, client.GetCluster().GetNodes().Count);
+
             ClientReconnectEventArgs eventArgs = null;
 
             client.ClientReconnected += (sender, args) => { eventArgs = args; };


[27/50] [abbrv] ignite git commit: Web console beta-7.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/generator-optional.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/generator-optional.js b/modules/web-console/frontend/app/modules/configuration/generator/generator-optional.js
deleted file mode 100644
index 61de1a2..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/generator-optional.js
+++ /dev/null
@@ -1,25 +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.
- */
-
-// Optional content generation entry point.
-const $generatorOptional = {};
-
-$generatorOptional.optionalContent = function(zip, cluster) { // eslint-disable-line no-unused-vars
-    // No-op.
-};
-
-export default $generatorOptional;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/generator-spring.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/generator-spring.js b/modules/web-console/frontend/app/modules/configuration/generator/generator-spring.js
deleted file mode 100644
index f70c66f..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/generator-spring.js
+++ /dev/null
@@ -1,2111 +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.
- */
-
-// XML generation entry point.
-const $generatorSpring = {};
-
-// Do XML escape.
-$generatorSpring.escape = function(s) {
-    if (typeof (s) !== 'string')
-        return s;
-
-    return s.replace(/&/g, '&amp;').replace(/</g, '&lt;').replace(/>/g, '&gt;').replace(/"/g, '&quot;');
-};
-
-// Add constructor argument
-$generatorSpring.constructorArg = function(res, ix, obj, propName, dflt, opt) {
-    const v = (obj ? obj[propName] : null) || dflt;
-
-    if ($generatorCommon.isDefinedAndNotEmpty(v))
-        res.line('<constructor-arg ' + (ix >= 0 ? 'index="' + ix + '" ' : '') + 'value="' + v + '"/>');
-    else if (!opt) {
-        res.startBlock('<constructor-arg ' + (ix >= 0 ? 'index="' + ix + '"' : '') + '>');
-        res.line('<null/>');
-        res.endBlock('</constructor-arg>');
-    }
-};
-
-// Add XML element.
-$generatorSpring.element = function(res, tag, attr1, val1, attr2, val2) {
-    let elem = '<' + tag;
-
-    if (attr1)
-        elem += ' ' + attr1 + '="' + val1 + '"';
-
-    if (attr2)
-        elem += ' ' + attr2 + '="' + val2 + '"';
-
-    elem += '/>';
-
-    res.emptyLineIfNeeded();
-    res.line(elem);
-};
-
-// Add property.
-$generatorSpring.property = function(res, obj, propName, setterName, dflt) {
-    if (!_.isNil(obj)) {
-        const val = obj[propName];
-
-        if ($generatorCommon.isDefinedAndNotEmpty(val)) {
-            const missDflt = _.isNil(dflt);
-
-            // Add to result if no default provided or value not equals to default.
-            if (missDflt || (!missDflt && val !== dflt)) {
-                $generatorSpring.element(res, 'property', 'name', setterName ? setterName : propName, 'value', $generatorSpring.escape(val));
-
-                return true;
-            }
-        }
-    }
-
-    return false;
-};
-
-// Add property for class name.
-$generatorSpring.classNameProperty = function(res, obj, propName) {
-    const val = obj[propName];
-
-    if (!_.isNil(val))
-        $generatorSpring.element(res, 'property', 'name', propName, 'value', $generatorCommon.JavaTypes.fullClassName(val));
-};
-
-// Add list property.
-$generatorSpring.listProperty = function(res, obj, propName, listType, rowFactory) {
-    const val = obj[propName];
-
-    if (val && val.length > 0) {
-        res.emptyLineIfNeeded();
-
-        if (!listType)
-            listType = 'list';
-
-        if (!rowFactory)
-            rowFactory = (v) => '<value>' + $generatorSpring.escape(v) + '</value>';
-
-        res.startBlock('<property name="' + propName + '">');
-        res.startBlock('<' + listType + '>');
-
-        _.forEach(val, (v) => res.line(rowFactory(v)));
-
-        res.endBlock('</' + listType + '>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-};
-
-// Add array property
-$generatorSpring.arrayProperty = function(res, obj, propName, descr, rowFactory) {
-    const val = obj[propName];
-
-    if (val && val.length > 0) {
-        res.emptyLineIfNeeded();
-
-        if (!rowFactory)
-            rowFactory = (v) => '<bean class="' + v + '"/>';
-
-        res.startBlock('<property name="' + propName + '">');
-        res.startBlock('<list>');
-
-        _.forEach(val, (v) => res.append(rowFactory(v)));
-
-        res.endBlock('</list>');
-        res.endBlock('</property>');
-    }
-};
-
-/**
- * Add bean property with internal content.
- *
- * @param res Optional configuration presentation builder object.
- * @param bean Bean object for code generation.
- * @param beanPropName Name of property to set generated bean as value.
- * @param desc Bean metadata object.
- * @param createBeanAlthoughNoProps Always generate bean even it has no properties defined.
- */
-$generatorSpring.beanProperty = function(res, bean, beanPropName, desc, createBeanAlthoughNoProps) {
-    const props = desc.fields;
-
-    if (bean && $generatorCommon.hasProperty(bean, props)) {
-        if (!createBeanAlthoughNoProps)
-            res.startSafeBlock();
-
-        res.emptyLineIfNeeded();
-        res.startBlock('<property name="' + beanPropName + '">');
-
-        if (createBeanAlthoughNoProps)
-            res.startSafeBlock();
-
-        res.startBlock('<bean class="' + desc.className + '">');
-
-        let hasData = false;
-
-        _.forIn(props, function(descr, propName) {
-            if (props.hasOwnProperty(propName)) {
-                if (descr) {
-                    switch (descr.type) {
-                        case 'list':
-                            $generatorSpring.listProperty(res, bean, propName, descr.setterName);
-
-                            break;
-
-                        case 'array':
-                            $generatorSpring.arrayProperty(res, bean, propName, descr);
-
-                            break;
-
-                        case 'propertiesAsList':
-                            const val = bean[propName];
-
-                            if (val && val.length > 0) {
-                                res.startBlock('<property name="' + propName + '">');
-                                res.startBlock('<props>');
-
-                                _.forEach(val, function(nameAndValue) {
-                                    const eqIndex = nameAndValue.indexOf('=');
-                                    if (eqIndex >= 0) {
-                                        res.line('<prop key="' + $generatorSpring.escape(nameAndValue.substring(0, eqIndex)) + '">' +
-                                            $generatorSpring.escape(nameAndValue.substr(eqIndex + 1)) + '</prop>');
-                                    }
-                                });
-
-                                res.endBlock('</props>');
-                                res.endBlock('</property>');
-
-                                hasData = true;
-                            }
-
-                            break;
-
-                        case 'bean':
-                            if ($generatorCommon.isDefinedAndNotEmpty(bean[propName])) {
-                                res.startBlock('<property name="' + propName + '">');
-                                res.line('<bean class="' + bean[propName] + '"/>');
-                                res.endBlock('</property>');
-
-                                hasData = true;
-                            }
-
-                            break;
-
-                        default:
-                            if ($generatorSpring.property(res, bean, propName, descr.setterName, descr.dflt))
-                                hasData = true;
-                    }
-                }
-                else
-                    if ($generatorSpring.property(res, bean, propName))
-                        hasData = true;
-            }
-        });
-
-        res.endBlock('</bean>');
-
-        if (createBeanAlthoughNoProps && !hasData) {
-            res.rollbackSafeBlock();
-
-            res.line('<bean class="' + desc.className + '"/>');
-        }
-
-        res.endBlock('</property>');
-
-        if (!createBeanAlthoughNoProps && !hasData)
-            res.rollbackSafeBlock();
-    }
-    else if (createBeanAlthoughNoProps) {
-        res.emptyLineIfNeeded();
-        res.startBlock('<property name="' + beanPropName + '">');
-        res.line('<bean class="' + desc.className + '"/>');
-        res.endBlock('</property>');
-    }
-};
-
-/**
- * Add bean property without internal content.
- *
- * @param res Optional configuration presentation builder object.
- * @param obj Object to take bean class name.
- * @param propName Property name.
- */
-$generatorSpring.simpleBeanProperty = function(res, obj, propName) {
-    if (!_.isNil(obj)) {
-        const val = obj[propName];
-
-        if ($generatorCommon.isDefinedAndNotEmpty(val)) {
-            res.startBlock('<property name="' + propName + '">');
-            res.line('<bean class="' + val + '"/>');
-            res.endBlock('</property>');
-        }
-    }
-
-    return false;
-};
-
-// Generate eviction policy.
-$generatorSpring.evictionPolicy = function(res, evtPlc, propName) {
-    if (evtPlc && evtPlc.kind) {
-        $generatorSpring.beanProperty(res, evtPlc[evtPlc.kind.toUpperCase()], propName,
-            $generatorCommon.EVICTION_POLICIES[evtPlc.kind], true);
-    }
-};
-
-// Generate discovery.
-$generatorSpring.clusterGeneral = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, cluster, 'name', 'gridName');
-    $generatorSpring.property(res, cluster, 'localHost');
-
-    if (cluster.discovery) {
-        res.startBlock('<property name="discoverySpi">');
-        res.startBlock('<bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">');
-        res.startBlock('<property name="ipFinder">');
-
-        const d = cluster.discovery;
-
-        switch (d.kind) {
-            case 'Multicast':
-                res.startBlock('<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">');
-
-                if (d.Multicast) {
-                    $generatorSpring.property(res, d.Multicast, 'multicastGroup');
-                    $generatorSpring.property(res, d.Multicast, 'multicastPort');
-                    $generatorSpring.property(res, d.Multicast, 'responseWaitTime');
-                    $generatorSpring.property(res, d.Multicast, 'addressRequestAttempts');
-                    $generatorSpring.property(res, d.Multicast, 'localAddress');
-                    $generatorSpring.listProperty(res, d.Multicast, 'addresses');
-                }
-
-                res.endBlock('</bean>');
-
-                break;
-
-            case 'Vm':
-                res.startBlock('<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">');
-
-                if (d.Vm)
-                    $generatorSpring.listProperty(res, d.Vm, 'addresses');
-
-                res.endBlock('</bean>');
-
-                break;
-
-            case 'S3':
-                res.startBlock('<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.s3.TcpDiscoveryS3IpFinder">');
-
-                if (d.S3) {
-                    if (d.S3.bucketName)
-                        res.line('<property name="bucketName" value="' + $generatorSpring.escape(d.S3.bucketName) + '"/>');
-                }
-
-                res.endBlock('</bean>');
-
-                break;
-
-            case 'Cloud':
-                res.startBlock('<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.cloud.TcpDiscoveryCloudIpFinder">');
-
-                if (d.Cloud) {
-                    $generatorSpring.property(res, d.Cloud, 'credential');
-                    $generatorSpring.property(res, d.Cloud, 'credentialPath');
-                    $generatorSpring.property(res, d.Cloud, 'identity');
-                    $generatorSpring.property(res, d.Cloud, 'provider');
-                    $generatorSpring.listProperty(res, d.Cloud, 'regions');
-                    $generatorSpring.listProperty(res, d.Cloud, 'zones');
-                }
-
-                res.endBlock('</bean>');
-
-                break;
-
-            case 'GoogleStorage':
-                res.startBlock('<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.gce.TcpDiscoveryGoogleStorageIpFinder">');
-
-                if (d.GoogleStorage) {
-                    $generatorSpring.property(res, d.GoogleStorage, 'projectName');
-                    $generatorSpring.property(res, d.GoogleStorage, 'bucketName');
-                    $generatorSpring.property(res, d.GoogleStorage, 'serviceAccountP12FilePath');
-                    $generatorSpring.property(res, d.GoogleStorage, 'serviceAccountId');
-                }
-
-                res.endBlock('</bean>');
-
-                break;
-
-            case 'Jdbc':
-                res.startBlock('<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinder">');
-
-                if (d.Jdbc) {
-                    const datasource = d.Jdbc;
-
-                    res.line('<property name="initSchema" value="' + (!_.isNil(datasource.initSchema) && datasource.initSchema) + '"/>');
-
-                    if (datasource.dataSourceBean && datasource.dialect) {
-                        res.line('<property name="dataSource" ref="' + datasource.dataSourceBean + '"/>');
-
-                        if (!_.find(res.datasources, { dataSourceBean: datasource.dataSourceBean })) {
-                            res.datasources.push({
-                                dataSourceBean: datasource.dataSourceBean,
-                                dialect: datasource.dialect
-                            });
-                        }
-                    }
-                }
-
-                res.endBlock('</bean>');
-
-                break;
-
-            case 'SharedFs':
-                res.startBlock('<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.sharedfs.TcpDiscoverySharedFsIpFinder">');
-
-                if (d.SharedFs)
-                    $generatorSpring.property(res, d.SharedFs, 'path');
-
-                res.endBlock('</bean>');
-
-                break;
-
-            case 'ZooKeeper':
-                res.startBlock('<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.zk.TcpDiscoveryZookeeperIpFinder">');
-
-                if (d.ZooKeeper) {
-                    if ($generatorCommon.isDefinedAndNotEmpty(d.ZooKeeper.curator)) {
-                        res.startBlock('<property name="curator">');
-                        res.line('<bean class="' + d.ZooKeeper.curator + '"/>');
-                        res.endBlock('</property>');
-                    }
-
-                    $generatorSpring.property(res, d.ZooKeeper, 'zkConnectionString');
-
-                    if (d.ZooKeeper.retryPolicy && d.ZooKeeper.retryPolicy.kind) {
-                        const kind = d.ZooKeeper.retryPolicy.kind;
-                        const retryPolicy = d.ZooKeeper.retryPolicy[kind];
-                        const customClassDefined = retryPolicy && $generatorCommon.isDefinedAndNotEmpty(retryPolicy.className);
-
-                        if (kind !== 'Custom' || customClassDefined)
-                            res.startBlock('<property name="retryPolicy">');
-
-                        switch (kind) {
-                            case 'ExponentialBackoff':
-                                res.startBlock('<bean class="org.apache.curator.retry.ExponentialBackoffRetry">');
-                                $generatorSpring.constructorArg(res, 0, retryPolicy, 'baseSleepTimeMs', 1000);
-                                $generatorSpring.constructorArg(res, 1, retryPolicy, 'maxRetries', 10);
-                                $generatorSpring.constructorArg(res, 2, retryPolicy, 'maxSleepMs', null, true);
-                                res.endBlock('</bean>');
-
-                                break;
-
-                            case 'BoundedExponentialBackoff':
-                                res.startBlock('<bean class="org.apache.curator.retry.BoundedExponentialBackoffRetry">');
-                                $generatorSpring.constructorArg(res, 0, retryPolicy, 'baseSleepTimeMs', 1000);
-                                $generatorSpring.constructorArg(res, 1, retryPolicy, 'maxSleepTimeMs', 2147483647);
-                                $generatorSpring.constructorArg(res, 2, retryPolicy, 'maxRetries', 10);
-                                res.endBlock('</bean>');
-
-                                break;
-
-                            case 'UntilElapsed':
-                                res.startBlock('<bean class="org.apache.curator.retry.RetryUntilElapsed">');
-                                $generatorSpring.constructorArg(res, 0, retryPolicy, 'maxElapsedTimeMs', 60000);
-                                $generatorSpring.constructorArg(res, 1, retryPolicy, 'sleepMsBetweenRetries', 1000);
-                                res.endBlock('</bean>');
-
-                                break;
-
-                            case 'NTimes':
-                                res.startBlock('<bean class="org.apache.curator.retry.RetryNTimes">');
-                                $generatorSpring.constructorArg(res, 0, retryPolicy, 'n', 10);
-                                $generatorSpring.constructorArg(res, 1, retryPolicy, 'sleepMsBetweenRetries', 1000);
-                                res.endBlock('</bean>');
-
-                                break;
-
-                            case 'OneTime':
-                                res.startBlock('<bean class="org.apache.curator.retry.RetryOneTime">');
-                                $generatorSpring.constructorArg(res, 0, retryPolicy, 'sleepMsBetweenRetry', 1000);
-                                res.endBlock('</bean>');
-
-                                break;
-
-                            case 'Forever':
-                                res.startBlock('<bean class="org.apache.curator.retry.RetryForever">');
-                                $generatorSpring.constructorArg(res, 0, retryPolicy, 'retryIntervalMs', 1000);
-                                res.endBlock('</bean>');
-
-                                break;
-
-                            case 'Custom':
-                                if (customClassDefined)
-                                    res.line('<bean class="' + retryPolicy.className + '"/>');
-
-                                break;
-
-                            default:
-                        }
-
-                        if (kind !== 'Custom' || customClassDefined)
-                            res.endBlock('</property>');
-                    }
-
-                    $generatorSpring.property(res, d.ZooKeeper, 'basePath', null, '/services');
-                    $generatorSpring.property(res, d.ZooKeeper, 'serviceName', null, 'ignite');
-                    $generatorSpring.property(res, d.ZooKeeper, 'allowDuplicateRegistrations', null, false);
-                }
-
-                res.endBlock('</bean>');
-
-                break;
-
-            default:
-                res.line('Unknown discovery kind: ' + d.kind);
-        }
-
-        res.endBlock('</property>');
-
-        $generatorSpring.clusterDiscovery(d, res);
-
-        res.endBlock('</bean>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate atomics group.
-$generatorSpring.clusterAtomics = function(atomics, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.hasAtLeastOneProperty(atomics, ['cacheMode', 'atomicSequenceReserveSize', 'backups'])) {
-        res.startSafeBlock();
-
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="atomicConfiguration">');
-        res.startBlock('<bean class="org.apache.ignite.configuration.AtomicConfiguration">');
-
-        const cacheMode = atomics.cacheMode ? atomics.cacheMode : 'PARTITIONED';
-
-        let hasData = cacheMode !== 'PARTITIONED';
-
-        $generatorSpring.property(res, atomics, 'cacheMode', null, 'PARTITIONED');
-
-        hasData = $generatorSpring.property(res, atomics, 'atomicSequenceReserveSize', null, 1000) || hasData;
-
-        if (cacheMode === 'PARTITIONED')
-            hasData = $generatorSpring.property(res, atomics, 'backups', null, 0) || hasData;
-
-        res.endBlock('</bean>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-
-        if (!hasData)
-            res.rollbackSafeBlock();
-    }
-
-    return res;
-};
-
-// Generate binary group.
-$generatorSpring.clusterBinary = function(binary, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.binaryIsDefined(binary)) {
-        res.startBlock('<property name="binaryConfiguration">');
-        res.startBlock('<bean class="org.apache.ignite.configuration.BinaryConfiguration">');
-
-        $generatorSpring.simpleBeanProperty(res, binary, 'idMapper');
-        $generatorSpring.simpleBeanProperty(res, binary, 'nameMapper');
-        $generatorSpring.simpleBeanProperty(res, binary, 'serializer');
-
-        if ($generatorCommon.isDefinedAndNotEmpty(binary.typeConfigurations)) {
-            res.startBlock('<property name="typeConfigurations">');
-            res.startBlock('<list>');
-
-            _.forEach(binary.typeConfigurations, function(type) {
-                res.startBlock('<bean class="org.apache.ignite.binary.BinaryTypeConfiguration">');
-
-                $generatorSpring.property(res, type, 'typeName');
-                $generatorSpring.simpleBeanProperty(res, type, 'idMapper');
-                $generatorSpring.simpleBeanProperty(res, type, 'nameMapper');
-                $generatorSpring.simpleBeanProperty(res, type, 'serializer');
-                $generatorSpring.property(res, type, 'enum', null, false);
-
-                res.endBlock('</bean>');
-            });
-
-            res.endBlock('</list>');
-            res.endBlock('</property>');
-        }
-
-        $generatorSpring.property(res, binary, 'compactFooter', null, true);
-
-        res.endBlock('</bean>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate cache key configurations.
-$generatorSpring.clusterCacheKeyConfiguration = function(keyCfgs, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    keyCfgs = _.filter(keyCfgs, (cfg) => cfg.typeName && cfg.affinityKeyFieldName);
-
-    if (_.isEmpty(keyCfgs))
-        return res;
-
-    res.startBlock('<property name="cacheKeyConfiguration">');
-    res.startBlock('<array>');
-
-    _.forEach(keyCfgs, (cfg) => {
-        res.startBlock('<bean class="org.apache.ignite.cache.CacheKeyConfiguration">');
-
-        $generatorSpring.constructorArg(res, -1, cfg, 'typeName');
-        $generatorSpring.constructorArg(res, -1, cfg, 'affinityKeyFieldName');
-
-        res.endBlock('</bean>');
-    });
-
-    res.endBlock('</array>');
-    res.endBlock('</property>');
-
-    return res;
-};
-
-// Generate collision group.
-$generatorSpring.clusterCollision = function(collision, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (collision && collision.kind && collision.kind !== 'Noop') {
-        const spi = collision[collision.kind];
-
-        if (collision.kind !== 'Custom' || (spi && $generatorCommon.isDefinedAndNotEmpty(spi.class))) {
-            res.startBlock('<property name="collisionSpi">');
-
-            switch (collision.kind) {
-                case 'JobStealing':
-                    res.startBlock('<bean class="org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi">');
-                    $generatorSpring.property(res, spi, 'activeJobsThreshold', null, 95);
-                    $generatorSpring.property(res, spi, 'waitJobsThreshold', null, 0);
-                    $generatorSpring.property(res, spi, 'messageExpireTime', null, 1000);
-                    $generatorSpring.property(res, spi, 'maximumStealingAttempts', null, 5);
-                    $generatorSpring.property(res, spi, 'stealingEnabled', null, true);
-
-                    if ($generatorCommon.isDefinedAndNotEmpty(spi.externalCollisionListener)) {
-                        res.needEmptyLine = true;
-
-                        res.startBlock('<property name="externalCollisionListener">');
-                        res.line('<bean class="' + spi.externalCollisionListener + ' "/>');
-                        res.endBlock('</property>');
-                    }
-
-                    if ($generatorCommon.isDefinedAndNotEmpty(spi.stealingAttributes)) {
-                        res.needEmptyLine = true;
-
-                        res.startBlock('<property name="stealingAttributes">');
-                        res.startBlock('<map>');
-
-                        _.forEach(spi.stealingAttributes, function(attr) {
-                            $generatorSpring.element(res, 'entry', 'key', attr.name, 'value', attr.value);
-                        });
-
-                        res.endBlock('</map>');
-                        res.endBlock('</property>');
-                    }
-
-                    res.endBlock('</bean>');
-
-                    break;
-
-                case 'FifoQueue':
-                    res.startBlock('<bean class="org.apache.ignite.spi.collision.fifoqueue.FifoQueueCollisionSpi">');
-                    $generatorSpring.property(res, spi, 'parallelJobsNumber');
-                    $generatorSpring.property(res, spi, 'waitingJobsNumber');
-                    res.endBlock('</bean>');
-
-                    break;
-
-                case 'PriorityQueue':
-                    res.startBlock('<bean class="org.apache.ignite.spi.collision.priorityqueue.PriorityQueueCollisionSpi">');
-                    $generatorSpring.property(res, spi, 'parallelJobsNumber');
-                    $generatorSpring.property(res, spi, 'waitingJobsNumber');
-                    $generatorSpring.property(res, spi, 'priorityAttributeKey', null, 'grid.task.priority');
-                    $generatorSpring.property(res, spi, 'jobPriorityAttributeKey', null, 'grid.job.priority');
-                    $generatorSpring.property(res, spi, 'defaultPriority', null, 0);
-                    $generatorSpring.property(res, spi, 'starvationIncrement', null, 1);
-                    $generatorSpring.property(res, spi, 'starvationPreventionEnabled', null, true);
-                    res.endBlock('</bean>');
-
-                    break;
-
-                case 'Custom':
-                    res.line('<bean class="' + spi.class + '"/>');
-
-                    break;
-
-                default:
-            }
-
-            res.endBlock('</property>');
-        }
-    }
-
-    return res;
-};
-
-// Generate communication group.
-$generatorSpring.clusterCommunication = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.beanProperty(res, cluster.communication, 'communicationSpi', $generatorCommon.COMMUNICATION_CONFIGURATION);
-
-    $generatorSpring.property(res, cluster, 'networkTimeout', null, 5000);
-    $generatorSpring.property(res, cluster, 'networkSendRetryDelay', null, 1000);
-    $generatorSpring.property(res, cluster, 'networkSendRetryCount', null, 3);
-    $generatorSpring.property(res, cluster, 'segmentCheckFrequency');
-    $generatorSpring.property(res, cluster, 'waitForSegmentOnStart', null, false);
-    $generatorSpring.property(res, cluster, 'discoveryStartupDelay', null, 60000);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-/**
- * XML generator for cluster's REST access configuration.
- *
- * @param connector Cluster REST connector configuration.
- * @param res Optional configuration presentation builder object.
- * @returns Configuration presentation builder object
- */
-$generatorSpring.clusterConnector = function(connector, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!_.isNil(connector) && connector.enabled) {
-        const cfg = _.cloneDeep($generatorCommon.CONNECTOR_CONFIGURATION);
-
-        if (connector.sslEnabled) {
-            cfg.fields.sslClientAuth = {dflt: false};
-            cfg.fields.sslFactory = {type: 'bean'};
-        }
-
-        $generatorSpring.beanProperty(res, connector, 'connectorConfiguration', cfg, true);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate deployment group.
-$generatorSpring.clusterDeployment = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorSpring.property(res, cluster, 'deploymentMode', null, 'SHARED'))
-        res.needEmptyLine = true;
-
-    const p2pEnabled = cluster.peerClassLoadingEnabled;
-
-    if (!_.isNil(p2pEnabled)) {
-        $generatorSpring.property(res, cluster, 'peerClassLoadingEnabled', null, false);
-
-        if (p2pEnabled) {
-            $generatorSpring.property(res, cluster, 'peerClassLoadingMissedResourcesCacheSize', null, 100);
-            $generatorSpring.property(res, cluster, 'peerClassLoadingThreadPoolSize', null, 2);
-            $generatorSpring.listProperty(res, cluster, 'peerClassLoadingLocalClassPathExclude');
-        }
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate discovery group.
-$generatorSpring.clusterDiscovery = function(disco, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (disco) {
-        $generatorSpring.property(res, disco, 'localAddress');
-        $generatorSpring.property(res, disco, 'localPort', null, 47500);
-        $generatorSpring.property(res, disco, 'localPortRange', null, 100);
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.addressResolver))
-            $generatorSpring.beanProperty(res, disco, 'addressResolver', {className: disco.addressResolver}, true);
-        $generatorSpring.property(res, disco, 'socketTimeout', null, 5000);
-        $generatorSpring.property(res, disco, 'ackTimeout', null, 5000);
-        $generatorSpring.property(res, disco, 'maxAckTimeout', null, 600000);
-        $generatorSpring.property(res, disco, 'networkTimeout', null, 5000);
-        $generatorSpring.property(res, disco, 'joinTimeout', null, 0);
-        $generatorSpring.property(res, disco, 'threadPriority', null, 10);
-        $generatorSpring.property(res, disco, 'heartbeatFrequency', null, 2000);
-        $generatorSpring.property(res, disco, 'maxMissedHeartbeats', null, 1);
-        $generatorSpring.property(res, disco, 'maxMissedClientHeartbeats', null, 5);
-        $generatorSpring.property(res, disco, 'topHistorySize', null, 1000);
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.listener))
-            $generatorSpring.beanProperty(res, disco, 'listener', {className: disco.listener}, true);
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.dataExchange))
-            $generatorSpring.beanProperty(res, disco, 'dataExchange', {className: disco.dataExchange}, true);
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.metricsProvider))
-            $generatorSpring.beanProperty(res, disco, 'metricsProvider', {className: disco.metricsProvider}, true);
-        $generatorSpring.property(res, disco, 'reconnectCount', null, 10);
-        $generatorSpring.property(res, disco, 'statisticsPrintFrequency', null, 0);
-        $generatorSpring.property(res, disco, 'ipFinderCleanFrequency', null, 60000);
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.authenticator))
-            $generatorSpring.beanProperty(res, disco, 'authenticator', {className: disco.authenticator}, true);
-        $generatorSpring.property(res, disco, 'forceServerMode', null, false);
-        $generatorSpring.property(res, disco, 'clientReconnectDisabled', null, false);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate events group.
-$generatorSpring.clusterEvents = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (cluster.includeEventTypes && cluster.includeEventTypes.length > 0) {
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="includeEventTypes">');
-
-        const evtGrps = angular.element(document.getElementById('app')).injector().get('igniteEventGroups');
-
-        if (cluster.includeEventTypes.length === 1) {
-            const evtGrp = _.find(evtGrps, {value: cluster.includeEventTypes[0]});
-
-            if (evtGrp)
-                res.line('<util:constant static-field="' + evtGrp.class + '.' + evtGrp.value + '"/>');
-        }
-        else {
-            res.startBlock('<list>');
-
-            _.forEach(cluster.includeEventTypes, (item, ix) => {
-                ix > 0 && res.line();
-
-                const evtGrp = _.find(evtGrps, {value: item});
-
-                if (evtGrp) {
-                    res.line('<!-- EventType.' + item + ' -->');
-
-                    _.forEach(evtGrp.events, (event) => res.line('<util:constant static-field="' + evtGrp.class + '.' + event + '"/>'));
-                }
-            });
-
-            res.endBlock('</list>');
-        }
-
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate failover group.
-$generatorSpring.clusterFailover = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.isDefinedAndNotEmpty(cluster.failoverSpi) && _.findIndex(cluster.failoverSpi, function(spi) {
-        return $generatorCommon.isDefinedAndNotEmpty(spi.kind) && (spi.kind !== 'Custom' || $generatorCommon.isDefinedAndNotEmpty(_.get(spi, spi.kind + '.class')));
-    }) >= 0) {
-        res.startBlock('<property name="failoverSpi">');
-        res.startBlock('<list>');
-
-        _.forEach(cluster.failoverSpi, function(spi) {
-            if (spi.kind && (spi.kind !== 'Custom' || $generatorCommon.isDefinedAndNotEmpty(_.get(spi, spi.kind + '.class')))) {
-                const maxAttempts = _.get(spi, spi.kind + '.maximumFailoverAttempts');
-
-                if ((spi.kind === 'JobStealing' || spi.kind === 'Always') && $generatorCommon.isDefinedAndNotEmpty(maxAttempts) && maxAttempts !== 5) {
-                    res.startBlock('<bean class="' + $generatorCommon.failoverSpiClass(spi) + '">');
-
-                    $generatorSpring.property(res, spi[spi.kind], 'maximumFailoverAttempts', null, 5);
-
-                    res.endBlock('</bean>');
-                }
-                else
-                    res.line('<bean class="' + $generatorCommon.failoverSpiClass(spi) + '"/>');
-
-                res.needEmptyLine = true;
-            }
-        });
-
-        res.needEmptyLine = true;
-
-        res.endBlock('</list>');
-        res.endBlock('</property>');
-    }
-
-    return res;
-};
-
-// Generate marshaller group.
-$generatorSpring.clusterLogger = function(logger, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.loggerConfigured(logger)) {
-        res.startBlock('<property name="gridLogger">');
-
-        const log = logger[logger.kind];
-
-        switch (logger.kind) {
-            case 'Log4j2':
-                res.startBlock('<bean class="org.apache.ignite.logger.log4j2.Log4J2Logger">');
-                res.line('<constructor-arg value="' + $generatorSpring.escape(log.path) + '"/>');
-                $generatorSpring.property(res, log, 'level');
-                res.endBlock('</bean>');
-
-                break;
-
-            case 'Null':
-                res.line('<bean class="org.apache.ignite.logger.NullLogger"/>');
-
-                break;
-
-            case 'Java':
-                res.line('<bean class="org.apache.ignite.logger.java.JavaLogger"/>');
-
-                break;
-
-            case 'JCL':
-                res.line('<bean class="org.apache.ignite.logger.jcl.JclLogger"/>');
-
-                break;
-
-            case 'SLF4J':
-                res.line('<bean class="org.apache.ignite.logger.slf4j.Slf4jLogger"/>');
-
-                break;
-
-            case 'Log4j':
-                if (log.mode === 'Default' && !$generatorCommon.isDefinedAndNotEmpty(log.level))
-                    res.line('<bean class="org.apache.ignite.logger.log4j.Log4JLogger"/>');
-                else {
-                    res.startBlock('<bean class="org.apache.ignite.logger.log4j.Log4JLogger">');
-
-                    if (log.mode === 'Path')
-                        res.line('<constructor-arg value="' + $generatorSpring.escape(log.path) + '"/>');
-
-                    $generatorSpring.property(res, log, 'level');
-                    res.endBlock('</bean>');
-                }
-
-                break;
-
-            case 'Custom':
-                res.line('<bean class="' + log.class + '"/>');
-
-                break;
-
-            default:
-        }
-
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate marshaller group.
-$generatorSpring.clusterMarshaller = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    const marshaller = cluster.marshaller;
-
-    if (marshaller && marshaller.kind)
-        $generatorSpring.beanProperty(res, marshaller[marshaller.kind], 'marshaller', $generatorCommon.MARSHALLERS[marshaller.kind], true);
-
-    res.softEmptyLine();
-
-    $generatorSpring.property(res, cluster, 'marshalLocalJobs', null, false);
-    $generatorSpring.property(res, cluster, 'marshallerCacheKeepAliveTime', null, 10000);
-    $generatorSpring.property(res, cluster, 'marshallerCacheThreadPoolSize', 'marshallerCachePoolSize');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate metrics group.
-$generatorSpring.clusterMetrics = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, cluster, 'metricsExpireTime');
-    $generatorSpring.property(res, cluster, 'metricsHistorySize', null, 10000);
-    $generatorSpring.property(res, cluster, 'metricsLogFrequency', null, 60000);
-    $generatorSpring.property(res, cluster, 'metricsUpdateFrequency', null, 2000);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate swap group.
-$generatorSpring.clusterSwap = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (cluster.swapSpaceSpi && cluster.swapSpaceSpi.kind === 'FileSwapSpaceSpi') {
-        $generatorSpring.beanProperty(res, cluster.swapSpaceSpi.FileSwapSpaceSpi, 'swapSpaceSpi',
-            $generatorCommon.SWAP_SPACE_SPI, true);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate time group.
-$generatorSpring.clusterTime = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, cluster, 'clockSyncSamples', null, 8);
-    $generatorSpring.property(res, cluster, 'clockSyncFrequency', null, 120000);
-    $generatorSpring.property(res, cluster, 'timeServerPortBase', null, 31100);
-    $generatorSpring.property(res, cluster, 'timeServerPortRange', null, 100);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate OBC configuration group.
-$generatorSpring.clusterODBC = function(odbc, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (odbc && odbc.odbcEnabled)
-        $generatorSpring.beanProperty(res, odbc, 'odbcConfiguration', $generatorCommon.ODBC_CONFIGURATION, true);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate thread pools group.
-$generatorSpring.clusterPools = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, cluster, 'publicThreadPoolSize');
-    $generatorSpring.property(res, cluster, 'systemThreadPoolSize');
-    $generatorSpring.property(res, cluster, 'managementThreadPoolSize');
-    $generatorSpring.property(res, cluster, 'igfsThreadPoolSize');
-    $generatorSpring.property(res, cluster, 'rebalanceThreadPoolSize');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate transactions group.
-$generatorSpring.clusterTransactions = function(transactionConfiguration, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.beanProperty(res, transactionConfiguration, 'transactionConfiguration', $generatorCommon.TRANSACTION_CONFIGURATION, false);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate user attributes group.
-$generatorSpring.clusterUserAttributes = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.isDefinedAndNotEmpty(cluster.attributes)) {
-        res.startBlock('<property name="userAttributes">');
-        res.startBlock('<map>');
-
-        _.forEach(cluster.attributes, function(attr) {
-            $generatorSpring.element(res, 'entry', 'key', attr.name, 'value', attr.value);
-        });
-
-        res.endBlock('</map>');
-        res.endBlock('</property>');
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-/**
- * XML generator for cluster's SSL configuration.
- *
- * @param cluster Cluster to get SSL configuration.
- * @param res Optional configuration presentation builder object.
- * @returns Configuration presentation builder object
- */
-$generatorSpring.clusterSsl = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (cluster.sslEnabled && !_.isNil(cluster.sslContextFactory)) {
-        let sslFactory;
-
-        if (_.isEmpty(cluster.sslContextFactory.keyStoreFilePath) && _.isEmpty(cluster.sslContextFactory.trustStoreFilePath))
-            sslFactory = cluster.sslContextFactory;
-        else {
-            sslFactory = _.clone(cluster.sslContextFactory);
-
-            sslFactory.keyStorePassword = _.isEmpty(cluster.sslContextFactory.keyStoreFilePath) ? null : '${ssl.key.storage.password}';
-            sslFactory.trustStorePassword = _.isEmpty(cluster.sslContextFactory.trustStoreFilePath) ? null : '${ssl.trust.storage.password}';
-        }
-
-        const propsDesc = $generatorCommon.isDefinedAndNotEmpty(cluster.sslContextFactory.trustManagers) ?
-            $generatorCommon.SSL_CONFIGURATION_TRUST_MANAGER_FACTORY :
-            $generatorCommon.SSL_CONFIGURATION_TRUST_FILE_FACTORY;
-
-        $generatorSpring.beanProperty(res, sslFactory, 'sslContextFactory', propsDesc, true);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate cache general group.
-$generatorSpring.cacheGeneral = function(cache, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, cache, 'name');
-
-    $generatorSpring.property(res, cache, 'cacheMode');
-    $generatorSpring.property(res, cache, 'atomicityMode');
-
-    if (cache.cacheMode === 'PARTITIONED' && $generatorSpring.property(res, cache, 'backups'))
-        $generatorSpring.property(res, cache, 'readFromBackup');
-
-    $generatorSpring.property(res, cache, 'copyOnRead');
-
-    if (cache.cacheMode === 'PARTITIONED' && cache.atomicityMode === 'TRANSACTIONAL')
-        $generatorSpring.property(res, cache, 'invalidate');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache memory group.
-$generatorSpring.cacheMemory = function(cache, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, cache, 'memoryMode', null, 'ONHEAP_TIERED');
-
-    if (cache.memoryMode !== 'OFFHEAP_VALUES')
-        $generatorSpring.property(res, cache, 'offHeapMaxMemory', null, -1);
-
-    res.softEmptyLine();
-
-    $generatorSpring.evictionPolicy(res, cache.evictionPolicy, 'evictionPolicy');
-
-    res.softEmptyLine();
-
-    $generatorSpring.property(res, cache, 'startSize', null, 1500000);
-    $generatorSpring.property(res, cache, 'swapEnabled', null, false);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache query & indexing group.
-$generatorSpring.cacheQuery = function(cache, domains, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, cache, 'sqlSchema');
-    $generatorSpring.property(res, cache, 'sqlOnheapRowCacheSize', null, 10240);
-    $generatorSpring.property(res, cache, 'longQueryWarningTimeout', null, 3000);
-
-    const indexedTypes = _.filter(domains, (domain) => domain.queryMetadata === 'Annotations');
-
-    if (indexedTypes.length > 0) {
-        res.softEmptyLine();
-
-        res.startBlock('<property name="indexedTypes">');
-        res.startBlock('<list>');
-
-        _.forEach(indexedTypes, function(domain) {
-            res.line('<value>' + $generatorCommon.JavaTypes.fullClassName(domain.keyType) + '</value>');
-            res.line('<value>' + $generatorCommon.JavaTypes.fullClassName(domain.valueType) + '</value>');
-        });
-
-        res.endBlock('</list>');
-        res.endBlock('</property>');
-    }
-
-    res.softEmptyLine();
-
-    $generatorSpring.listProperty(res, cache, 'sqlFunctionClasses');
-
-    res.softEmptyLine();
-
-    $generatorSpring.property(res, cache, 'snapshotableIndex', null, false);
-    $generatorSpring.property(res, cache, 'sqlEscapeAll', null, false);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache store group.
-$generatorSpring.cacheStore = function(cache, domains, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) {
-        const factoryKind = cache.cacheStoreFactory.kind;
-
-        const storeFactory = cache.cacheStoreFactory[factoryKind];
-
-        if (storeFactory) {
-            if (factoryKind === 'CacheJdbcPojoStoreFactory') {
-                res.startBlock('<property name="cacheStoreFactory">');
-                res.startBlock('<bean class="org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory">');
-
-                $generatorSpring.property(res, storeFactory, 'dataSourceBean');
-
-                res.startBlock('<property name="dialect">');
-                res.line('<bean class="' + $generatorCommon.jdbcDialectClassName(storeFactory.dialect) + '"/>');
-                res.endBlock('</property>');
-
-                if (storeFactory.sqlEscapeAll)
-                    $generatorSpring.property(res, storeFactory, 'sqlEscapeAll');
-
-                const domainConfigs = _.filter(domains, function(domain) {
-                    return $generatorCommon.isDefinedAndNotEmpty(domain.databaseTable);
-                });
-
-                if ($generatorCommon.isDefinedAndNotEmpty(domainConfigs)) {
-                    res.startBlock('<property name="types">');
-                    res.startBlock('<list>');
-
-                    _.forEach(domainConfigs, function(domain) {
-                        res.startBlock('<bean class="org.apache.ignite.cache.store.jdbc.JdbcType">');
-
-                        $generatorSpring.property(res, cache, 'name', 'cacheName');
-
-                        $generatorSpring.classNameProperty(res, domain, 'keyType');
-                        $generatorSpring.property(res, domain, 'valueType');
-
-                        $generatorSpring.domainStore(domain, res);
-
-                        res.endBlock('</bean>');
-                    });
-
-                    res.endBlock('</list>');
-                    res.endBlock('</property>');
-                }
-
-                res.endBlock('</bean>');
-                res.endBlock('</property>');
-            }
-            else if (factoryKind === 'CacheJdbcBlobStoreFactory') {
-                res.startBlock('<property name="cacheStoreFactory">');
-                res.startBlock('<bean class="org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactory">');
-
-                if (storeFactory.connectVia === 'DataSource')
-                    $generatorSpring.property(res, storeFactory, 'dataSourceBean');
-                else {
-                    $generatorSpring.property(res, storeFactory, 'connectionUrl');
-                    $generatorSpring.property(res, storeFactory, 'user');
-                    res.line('<property name="password" value="${ds.' + storeFactory.user + '.password}"/>');
-                }
-
-                $generatorSpring.property(res, storeFactory, 'initSchema');
-                $generatorSpring.property(res, storeFactory, 'createTableQuery');
-                $generatorSpring.property(res, storeFactory, 'loadQuery');
-                $generatorSpring.property(res, storeFactory, 'insertQuery');
-                $generatorSpring.property(res, storeFactory, 'updateQuery');
-                $generatorSpring.property(res, storeFactory, 'deleteQuery');
-
-                res.endBlock('</bean>');
-                res.endBlock('</property>');
-            }
-            else
-                $generatorSpring.beanProperty(res, storeFactory, 'cacheStoreFactory', $generatorCommon.STORE_FACTORIES[factoryKind], true);
-
-            if (storeFactory.dataSourceBean && (storeFactory.connectVia ? (storeFactory.connectVia === 'DataSource' ? storeFactory.dialect : null) : storeFactory.dialect)) {
-                if (!_.find(res.datasources, { dataSourceBean: storeFactory.dataSourceBean})) {
-                    res.datasources.push({
-                        dataSourceBean: storeFactory.dataSourceBean,
-                        dialect: storeFactory.dialect
-                    });
-                }
-            }
-        }
-    }
-
-    res.softEmptyLine();
-
-    $generatorSpring.property(res, cache, 'storeKeepBinary', null, false);
-    $generatorSpring.property(res, cache, 'loadPreviousValue', null, false);
-    $generatorSpring.property(res, cache, 'readThrough', null, false);
-    $generatorSpring.property(res, cache, 'writeThrough', null, false);
-
-    res.softEmptyLine();
-
-    if (cache.writeBehindEnabled) {
-        $generatorSpring.property(res, cache, 'writeBehindEnabled', null, false);
-        $generatorSpring.property(res, cache, 'writeBehindBatchSize', null, 512);
-        $generatorSpring.property(res, cache, 'writeBehindFlushSize', null, 10240);
-        $generatorSpring.property(res, cache, 'writeBehindFlushFrequency', null, 5000);
-        $generatorSpring.property(res, cache, 'writeBehindFlushThreadCount', null, 1);
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache node filter group.
-$generatorSpring.cacheNodeFilter = function(cache, igfss, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    const kind = _.get(cache, 'nodeFilter.kind');
-
-    if (_.isNil(kind) || _.isNil(cache.nodeFilter[kind]))
-        return res;
-
-    switch (kind) {
-        case 'IGFS':
-            const foundIgfs = _.find(igfss, (igfs) => igfs._id === cache.nodeFilter.IGFS.igfs);
-
-            if (foundIgfs) {
-                res.startBlock('<property name="nodeFilter">');
-                res.startBlock('<bean class="org.apache.ignite.internal.processors.igfs.IgfsNodePredicate">');
-                res.line('<constructor-arg value="' + foundIgfs.name + '"/>');
-                res.endBlock('</bean>');
-                res.endBlock('</property>');
-            }
-
-            break;
-
-        case 'OnNodes':
-            const nodes = cache.nodeFilter.OnNodes.nodeIds;
-
-            if ($generatorCommon.isDefinedAndNotEmpty(nodes)) {
-                res.startBlock('<property name="nodeFilter">');
-                res.startBlock('<bean class="org.apache.ignite.internal.util.lang.GridNodePredicate">');
-                res.startBlock('<constructor-arg>');
-                res.startBlock('<list>');
-
-                _.forEach(nodes, (nodeId) => {
-                    res.startBlock('<bean class="java.util.UUID" factory-method="fromString">');
-                    res.line('<constructor-arg value="' + nodeId + '"/>');
-                    res.endBlock('</bean>');
-                });
-
-                res.endBlock('</list>');
-                res.endBlock('</constructor-arg>');
-                res.endBlock('</bean>');
-                res.endBlock('</property>');
-            }
-
-            break;
-
-        case 'Custom':
-            res.startBlock('<property name="nodeFilter">');
-            res.line('<bean class="' + cache.nodeFilter.Custom.className + '"/>');
-            res.endBlock('</property>');
-
-            break;
-
-        default: break;
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache concurrency group.
-$generatorSpring.cacheConcurrency = function(cache, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, cache, 'maxConcurrentAsyncOperations', null, 500);
-    $generatorSpring.property(res, cache, 'defaultLockTimeout', null, 0);
-    $generatorSpring.property(res, cache, 'atomicWriteOrderMode');
-    $generatorSpring.property(res, cache, 'writeSynchronizationMode', null, 'PRIMARY_SYNC');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache rebalance group.
-$generatorSpring.cacheRebalance = function(cache, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (cache.cacheMode !== 'LOCAL') {
-        $generatorSpring.property(res, cache, 'rebalanceMode', null, 'ASYNC');
-        $generatorSpring.property(res, cache, 'rebalanceThreadPoolSize', null, 1);
-        $generatorSpring.property(res, cache, 'rebalanceBatchSize', null, 524288);
-        $generatorSpring.property(res, cache, 'rebalanceBatchesPrefetchCount', null, 2);
-        $generatorSpring.property(res, cache, 'rebalanceOrder', null, 0);
-        $generatorSpring.property(res, cache, 'rebalanceDelay', null, 0);
-        $generatorSpring.property(res, cache, 'rebalanceTimeout', null, 10000);
-        $generatorSpring.property(res, cache, 'rebalanceThrottle', null, 0);
-    }
-
-    res.softEmptyLine();
-
-    if (cache.igfsAffinnityGroupSize) {
-        res.startBlock('<property name="affinityMapper">');
-        res.startBlock('<bean class="org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper">');
-        $generatorSpring.constructorArg(res, -1, cache, 'igfsAffinnityGroupSize');
-        res.endBlock('</bean>');
-        res.endBlock('</property>');
-    }
-
-    return res;
-};
-
-// Generate cache server near cache group.
-$generatorSpring.cacheServerNearCache = function(cache, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (cache.cacheMode === 'PARTITIONED' && cache.nearCacheEnabled) {
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="nearConfiguration">');
-        res.startBlock('<bean class="org.apache.ignite.configuration.NearCacheConfiguration">');
-
-        if (cache.nearConfiguration) {
-            if (cache.nearConfiguration.nearStartSize)
-                $generatorSpring.property(res, cache.nearConfiguration, 'nearStartSize', null, 375000);
-
-            $generatorSpring.evictionPolicy(res, cache.nearConfiguration.nearEvictionPolicy, 'nearEvictionPolicy');
-        }
-
-        res.endBlock('</bean>');
-        res.endBlock('</property>');
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache statistics group.
-$generatorSpring.cacheStatistics = function(cache, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, cache, 'statisticsEnabled', null, false);
-    $generatorSpring.property(res, cache, 'managementEnabled', null, false);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate domain model query fields.
-$generatorSpring.domainModelQueryFields = function(res, domain) {
-    const fields = domain.fields;
-
-    if (fields && fields.length > 0) {
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="fields">');
-        res.startBlock('<map>');
-
-        _.forEach(fields, function(field) {
-            $generatorSpring.element(res, 'entry', 'key', field.name, 'value', $generatorCommon.JavaTypes.fullClassName(field.className));
-        });
-
-        res.endBlock('</map>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-};
-
-// Generate domain model query fields.
-$generatorSpring.domainModelQueryAliases = function(res, domain) {
-    const aliases = domain.aliases;
-
-    if (aliases && aliases.length > 0) {
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="aliases">');
-        res.startBlock('<map>');
-
-        _.forEach(aliases, function(alias) {
-            $generatorSpring.element(res, 'entry', 'key', alias.field, 'value', alias.alias);
-        });
-
-        res.endBlock('</map>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-};
-
-// Generate domain model indexes.
-$generatorSpring.domainModelQueryIndexes = function(res, domain) {
-    const indexes = domain.indexes;
-
-    if (indexes && indexes.length > 0) {
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="indexes">');
-        res.startBlock('<list>');
-
-        _.forEach(indexes, function(index) {
-            res.startBlock('<bean class="org.apache.ignite.cache.QueryIndex">');
-
-            $generatorSpring.property(res, index, 'name');
-            $generatorSpring.property(res, index, 'indexType');
-
-            const fields = index.fields;
-
-            if (fields && fields.length > 0) {
-                res.startBlock('<property name="fields">');
-                res.startBlock('<map>');
-
-                _.forEach(fields, function(field) {
-                    $generatorSpring.element(res, 'entry', 'key', field.name, 'value', field.direction);
-                });
-
-                res.endBlock('</map>');
-                res.endBlock('</property>');
-            }
-
-            res.endBlock('</bean>');
-        });
-
-        res.endBlock('</list>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-};
-
-// Generate domain model db fields.
-$generatorSpring.domainModelDatabaseFields = function(res, domain, fieldProp) {
-    const fields = domain[fieldProp];
-
-    if (fields && fields.length > 0) {
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="' + fieldProp + '">');
-
-        res.startBlock('<list>');
-
-        _.forEach(fields, function(field) {
-            res.startBlock('<bean class="org.apache.ignite.cache.store.jdbc.JdbcTypeField">');
-
-            $generatorSpring.property(res, field, 'databaseFieldName');
-
-            res.startBlock('<property name="databaseFieldType">');
-            res.line('<util:constant static-field="java.sql.Types.' + field.databaseFieldType + '"/>');
-            res.endBlock('</property>');
-
-            $generatorSpring.property(res, field, 'javaFieldName');
-
-            $generatorSpring.classNameProperty(res, field, 'javaFieldType');
-
-            res.endBlock('</bean>');
-        });
-
-        res.endBlock('</list>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-};
-
-// Generate domain model general group.
-$generatorSpring.domainModelGeneral = function(domain, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    switch ($generatorCommon.domainQueryMetadata(domain)) {
-        case 'Annotations':
-            if ($generatorCommon.isDefinedAndNotEmpty(domain.keyType) || $generatorCommon.isDefinedAndNotEmpty(domain.valueType)) {
-                res.startBlock('<property name="indexedTypes">');
-                res.startBlock('<list>');
-
-                if ($generatorCommon.isDefinedAndNotEmpty(domain.keyType))
-                    res.line('<value>' + $generatorCommon.JavaTypes.fullClassName(domain.keyType) + '</value>');
-                else
-                    res.line('<value>???</value>');
-
-                if ($generatorCommon.isDefinedAndNotEmpty(domain.valueType))
-                    res.line('<value>' + $generatorCommon.JavaTypes.fullClassName(domain.valueType) + '</value>');
-                else
-                    res.line('<value>>???</value>');
-
-                res.endBlock('</list>');
-                res.endBlock('</property>');
-            }
-
-            break;
-
-        case 'Configuration':
-            $generatorSpring.classNameProperty(res, domain, 'keyType');
-            $generatorSpring.property(res, domain, 'valueType');
-
-            break;
-
-        default:
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate domain model for query group.
-$generatorSpring.domainModelQuery = function(domain, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.domainQueryMetadata(domain) === 'Configuration') {
-        $generatorSpring.domainModelQueryFields(res, domain);
-        $generatorSpring.domainModelQueryAliases(res, domain);
-        $generatorSpring.domainModelQueryIndexes(res, domain);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate domain model for store group.
-$generatorSpring.domainStore = function(domain, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, domain, 'databaseSchema');
-    $generatorSpring.property(res, domain, 'databaseTable');
-
-    res.softEmptyLine();
-
-    $generatorSpring.domainModelDatabaseFields(res, domain, 'keyFields');
-    $generatorSpring.domainModelDatabaseFields(res, domain, 'valueFields');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-$generatorSpring.cacheQueryMetadata = function(domain, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    res.startBlock('<bean class="org.apache.ignite.cache.QueryEntity">');
-
-    $generatorSpring.classNameProperty(res, domain, 'keyType');
-    $generatorSpring.property(res, domain, 'valueType');
-
-    $generatorSpring.domainModelQuery(domain, res);
-
-    res.endBlock('</bean>');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate domain models configs.
-$generatorSpring.cacheDomains = function(domains, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    const domainConfigs = _.filter(domains, function(domain) {
-        return $generatorCommon.domainQueryMetadata(domain) === 'Configuration' &&
-            $generatorCommon.isDefinedAndNotEmpty(domain.fields);
-    });
-
-    if ($generatorCommon.isDefinedAndNotEmpty(domainConfigs)) {
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="queryEntities">');
-        res.startBlock('<list>');
-
-        _.forEach(domainConfigs, function(domain) {
-            $generatorSpring.cacheQueryMetadata(domain, res);
-        });
-
-        res.endBlock('</list>');
-        res.endBlock('</property>');
-    }
-
-    return res;
-};
-
-// Generate cache configs.
-$generatorSpring.cache = function(cache, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    res.startBlock('<bean class="org.apache.ignite.configuration.CacheConfiguration">');
-
-    $generatorSpring.cacheConfiguration(cache, res);
-
-    res.endBlock('</bean>');
-
-    return res;
-};
-
-// Generate cache configs.
-$generatorSpring.cacheConfiguration = function(cache, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.cacheGeneral(cache, res);
-    $generatorSpring.cacheMemory(cache, res);
-    $generatorSpring.cacheQuery(cache, cache.domains, res);
-    $generatorSpring.cacheStore(cache, cache.domains, res);
-
-    const igfs = _.get(cache, 'nodeFilter.IGFS.instance');
-
-    $generatorSpring.cacheNodeFilter(cache, igfs ? [igfs] : [], res);
-    $generatorSpring.cacheConcurrency(cache, res);
-    $generatorSpring.cacheRebalance(cache, res);
-    $generatorSpring.cacheServerNearCache(cache, res);
-    $generatorSpring.cacheStatistics(cache, res);
-    $generatorSpring.cacheDomains(cache.domains, res);
-
-    return res;
-};
-
-// Generate caches configs.
-$generatorSpring.clusterCaches = function(caches, igfss, isSrvCfg, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.isDefinedAndNotEmpty(caches) || (isSrvCfg && $generatorCommon.isDefinedAndNotEmpty(igfss))) {
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="cacheConfiguration">');
-        res.startBlock('<list>');
-
-        _.forEach(caches, function(cache) {
-            $generatorSpring.cache(cache, res);
-
-            res.needEmptyLine = true;
-        });
-
-        if (isSrvCfg) {
-            _.forEach(igfss, (igfs) => {
-                $generatorSpring.cache($generatorCommon.igfsDataCache(igfs), res);
-
-                res.needEmptyLine = true;
-
-                $generatorSpring.cache($generatorCommon.igfsMetaCache(igfs), res);
-
-                res.needEmptyLine = true;
-            });
-        }
-
-        res.endBlock('</list>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate IGFSs configs.
-$generatorSpring.igfss = function(igfss, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.isDefinedAndNotEmpty(igfss)) {
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="fileSystemConfiguration">');
-        res.startBlock('<list>');
-
-        _.forEach(igfss, function(igfs) {
-            res.startBlock('<bean class="org.apache.ignite.configuration.FileSystemConfiguration">');
-
-            $generatorSpring.igfsGeneral(igfs, res);
-            $generatorSpring.igfsIPC(igfs, res);
-            $generatorSpring.igfsFragmentizer(igfs, res);
-            $generatorSpring.igfsDualMode(igfs, res);
-            $generatorSpring.igfsSecondFS(igfs, res);
-            $generatorSpring.igfsMisc(igfs, res);
-
-            res.endBlock('</bean>');
-
-            res.needEmptyLine = true;
-        });
-
-        res.endBlock('</list>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate IGFS IPC configuration.
-$generatorSpring.igfsIPC = function(igfs, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (igfs.ipcEndpointEnabled) {
-        $generatorSpring.beanProperty(res, igfs.ipcEndpointConfiguration, 'ipcEndpointConfiguration', $generatorCommon.IGFS_IPC_CONFIGURATION, true);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate IGFS fragmentizer configuration.
-$generatorSpring.igfsFragmentizer = function(igfs, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (igfs.fragmentizerEnabled) {
-        $generatorSpring.property(res, igfs, 'fragmentizerConcurrentFiles', null, 0);
-        $generatorSpring.property(res, igfs, 'fragmentizerThrottlingBlockLength', null, 16777216);
-        $generatorSpring.property(res, igfs, 'fragmentizerThrottlingDelay', null, 200);
-
-        res.needEmptyLine = true;
-    }
-    else
-        $generatorSpring.property(res, igfs, 'fragmentizerEnabled');
-
-    return res;
-};
-
-// Generate IGFS dual mode configuration.
-$generatorSpring.igfsDualMode = function(igfs, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, igfs, 'dualModeMaxPendingPutsSize', null, 0);
-
-    if ($generatorCommon.isDefinedAndNotEmpty(igfs.dualModePutExecutorService)) {
-        res.startBlock('<property name="dualModePutExecutorService">');
-        res.line('<bean class="' + igfs.dualModePutExecutorService + '"/>');
-        res.endBlock('</property>');
-    }
-
-    $generatorSpring.property(res, igfs, 'dualModePutExecutorServiceShutdown', null, false);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-$generatorSpring.igfsSecondFS = function(igfs, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (igfs.secondaryFileSystemEnabled) {
-        const secondFs = igfs.secondaryFileSystem || {};
-
-        res.startBlock('<property name="secondaryFileSystem">');
-
-        res.startBlock('<bean class="org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem">');
-
-        const nameDefined = $generatorCommon.isDefinedAndNotEmpty(secondFs.userName);
-        const cfgDefined = $generatorCommon.isDefinedAndNotEmpty(secondFs.cfgPath);
-
-        $generatorSpring.constructorArg(res, 0, secondFs, 'uri');
-
-        if (cfgDefined || nameDefined)
-            $generatorSpring.constructorArg(res, 1, secondFs, 'cfgPath');
-
-        $generatorSpring.constructorArg(res, 2, secondFs, 'userName', null, true);
-
-        res.endBlock('</bean>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate IGFS general configuration.
-$generatorSpring.igfsGeneral = function(igfs, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.isDefinedAndNotEmpty(igfs.name)) {
-        igfs.dataCacheName = $generatorCommon.igfsDataCache(igfs).name;
-        igfs.metaCacheName = $generatorCommon.igfsMetaCache(igfs).name;
-
-        $generatorSpring.property(res, igfs, 'name');
-        $generatorSpring.property(res, igfs, 'dataCacheName');
-        $generatorSpring.property(res, igfs, 'metaCacheName');
-        $generatorSpring.property(res, igfs, 'defaultMode', null, 'DUAL_ASYNC');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate IGFS misc configuration.
-$generatorSpring.igfsMisc = function(igfs, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, igfs, 'blockSize', null, 65536);
-    $generatorSpring.property(res, igfs, 'streamBufferSize', null, 65536);
-    $generatorSpring.property(res, igfs, 'maxSpaceSize', null, 0);
-    $generatorSpring.property(res, igfs, 'maximumTaskRangeLength', null, 0);
-    $generatorSpring.property(res, igfs, 'managementPort', null, 11400);
-    $generatorSpring.property(res, igfs, 'perNodeBatchSize', null, 100);
-    $generatorSpring.property(res, igfs, 'perNodeParallelBatchCount', null, 8);
-    $generatorSpring.property(res, igfs, 'prefetchBlocks', null, 0);
-    $generatorSpring.property(res, igfs, 'sequentialReadsBeforePrefetch', null, 0);
-    $generatorSpring.property(res, igfs, 'trashPurgeTimeout', null, 1000);
-    $generatorSpring.property(res, igfs, 'colocateMetadata', null, true);
-    $generatorSpring.property(res, igfs, 'relaxedConsistency', null, true);
-
-    res.softEmptyLine();
-
-    if (igfs.pathModes && igfs.pathModes.length > 0) {
-        res.startBlock('<property name="pathModes">');
-        res.startBlock('<map>');
-
-        _.forEach(igfs.pathModes, function(pair) {
-            res.line('<entry key="' + pair.path + '" value="' + pair.mode + '"/>');
-        });
-
-        res.endBlock('</map>');
-        res.endBlock('</property>');
-    }
-
-    return res;
-};
-
-// Generate DataSource beans.
-$generatorSpring.generateDataSources = function(datasources, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (datasources.length > 0) {
-        res.line('<!-- Data source beans will be initialized from external properties file. -->');
-
-        _.forEach(datasources, (datasource) => $generatorSpring.generateDataSource(datasource, res));
-
-        res.needEmptyLine = true;
-
-        res.emptyLineIfNeeded();
-    }
-
-    return res;
-};
-
-$generatorSpring.generateDataSource = function(datasource, res) {
-    const beanId = datasource.dataSourceBean;
-
-    res.startBlock('<bean id="' + beanId + '" class="' + $generatorCommon.DATA_SOURCES[datasource.dialect] + '">');
-
-    switch (datasource.dialect) {
-        case 'Generic':
-            res.line('<property name="jdbcUrl" value="${' + beanId + '.jdbc.url}"/>');
-
-            break;
-
-        case 'DB2':
-            res.line('<property name="serverName" value="${' + beanId + '.jdbc.server_name}"/>');
-            res.line('<property name="portNumber" value="${' + beanId + '.jdbc.port_number}"/>');
-            res.line('<property name="databaseName" value="${' + beanId + '.jdbc.database_name}"/>');
-            res.line('<property name="driverType" value="${' + beanId + '.jdbc.driver_type}"/>');
-
-            break;
-
-        case 'PostgreSQL':
-            res.line('<property name="url" value="${' + beanId + '.jdbc.url}"/>');
-
-            break;
-
-        default:
-            res.line('<property name="URL" value="${' + beanId + '.jdbc.url}"/>');
-    }
-
-    res.line('<property name="user" value="${' + beanId + '.jdbc.username}"/>');
-    res.line('<property name="password" value="${' + beanId + '.jdbc.password}"/>');
-
-    res.endBlock('</bean>');
-
-    res.needEmptyLine = true;
-
-    res.emptyLineIfNeeded();
-};
-
-$generatorSpring.clusterConfiguration = function(cluster, clientNearCfg, res) {
-    const isSrvCfg = _.isNil(clientNearCfg);
-
-    if (!isSrvCfg) {
-        res.line('<property name="clientMode" value="true"/>');
-
-        res.needEmptyLine = true;
-    }
-
-    $generatorSpring.clusterGeneral(cluster, res);
-
-    $generatorSpring.clusterAtomics(cluster.atomicConfiguration, res);
-
-    $generatorSpring.clusterBinary(cluster.binaryConfiguration, res);
-
-    $generatorSpring.clusterCacheKeyConfiguration(cluster.cacheKeyConfiguration, res);
-
-    $generatorSpring.clusterCollision(cluster.collision, res);
-
-    $generatorSpring.clusterCommunication(cluster, res);
-
-    $generatorSpring.clusterConnector(cluster.connector, res);
-
-    $generatorSpring.clusterDeployment(cluster, res);
-
-    $generatorSpring.clusterEvents(cluster, res);
-
-    $generatorSpring.clusterFailover(cluster, res);
-
-    $generatorSpring.clusterLogger(cluster.logger, res);
-
-    $generatorSpring.clusterODBC(cluster.odbc, res);
-
-    $generatorSpring.clusterMarshaller(cluster, res);
-
-    $generatorSpring.clusterMetrics(cluster, res);
-
-    $generatorSpring.clusterSwap(cluster, res);
-
-    $generatorSpring.clusterTime(cluster, res);
-
-    $generatorSpring.clusterPools(cluster, res);
-
-    $generatorSpring.clusterTransactions(cluster.transactionConfiguration, res);
-
-    $generatorSpring.clusterCaches(cluster.caches, cluster.igfss, isSrvCfg, res);
-
-    $generatorSpring.clusterSsl(cluster, res);
-
-    if (isSrvCfg)
-        $generatorSpring.igfss(cluster.igfss, res);
-
-    $generatorSpring.clusterUserAttributes(cluster, res);
-
-    return res;
-};
-
-$generatorSpring.cluster = function(cluster, clientNearCfg) {
-    if (cluster) {
-        const res = $generatorCommon.builder(1);
-
-        if (clientNearCfg) {
-            res.startBlock('<bean id="nearCacheBean" class="org.apache.ignite.configuration.NearCacheConfiguration">');
-
-            if (clientNearCfg.nearStartSize)
-                $generatorSpring.property(res, clientNearCfg, 'nearStartSize');
-
-            if (clientNearCfg.nearEvictionPolicy && clientNearCfg.nearEvictionPolicy.kind)
-                $generatorSpring.evictionPolicy(res, clientNearCfg.nearEvictionPolicy, 'nearEvictionPolicy');
-
-            res.endBlock('</bean>');
-
-            res.needEmptyLine = true;
-
-            res.emptyLineIfNeeded();
-        }
-
-        // Generate Ignite Configuration.
-        res.startBlock('<bean class="org.apache.ignite.configuration.IgniteConfiguration">');
-
-        $generatorSpring.clusterConfiguration(cluster, clientNearCfg, res);
-
-        res.endBlock('</bean>');
-
-        // Build final XML:
-        // 1. Add header.
-        let xml = '<?xml version="1.0" encoding="UTF-8"?>\n\n';
-
-        xml += '<!-- ' + $generatorCommon.mainComment('configuration') + ' -->\n\n';
-        xml += '<beans xmlns="http://www.springframework.org/schema/beans"\n';
-        xml += '       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"\n';
-        xml += '       xmlns:util="http://www.springframework.org/schema/util"\n';
-        xml += '       xsi:schemaLocation="http://www.springframework.org/schema/beans\n';
-        xml += '                           http://www.springframework.org/schema/beans/spring-beans.xsd\n';
-        xml += '                           http://www.springframework.org/schema/util\n';
-        xml += '                           http://www.springframework.org/schema/util/spring-util.xsd">\n';
-
-        // 2. Add external property file
-        if ($generatorCommon.secretPropertiesNeeded(cluster)) {
-            xml += '    <!-- Load external properties file. -->\n';
-            xml += '    <bean id="placeholderConfig" class="org.springframework.beans.factory.config.PropertyPlaceholderConfigurer">\n';
-            xml += '        <property name="location" value="classpath:secret.properties"/>\n';
-            xml += '    </bean>\n\n';
-        }
-
-        // 3. Add data sources.
-        xml += $generatorSpring.generateDataSources(res.datasources, $generatorCommon.builder(1)).asString();
-
-        // 3. Add main content.
-        xml += res.asString();
-
-        // 4. Add footer.
-        xml += '\n</beans>';
-
-        return xml;
-    }
-
-    return '';
-};
-
-export default $generatorSpring;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/sql/Notebook.data.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/sql/Notebook.data.js b/modules/web-console/frontend/app/modules/sql/Notebook.data.js
index f66faba..3f98bed 100644
--- a/modules/web-console/frontend/app/modules/sql/Notebook.data.js
+++ b/modules/web-console/frontend/app/modules/sql/Notebook.data.js
@@ -21,7 +21,8 @@ const DEMO_NOTEBOOK = {
         {
             name: 'Query with refresh rate',
             cacheName: 'CarCache',
-            pageSize: 50,
+            pageSize: 100,
+            limit: 0,
             query: [
                 'SELECT count(*)',
                 'FROM "CarCache".Car'
@@ -37,7 +38,8 @@ const DEMO_NOTEBOOK = {
         {
             name: 'Simple query',
             cacheName: 'CarCache',
-            pageSize: 50,
+            pageSize: 100,
+            limit: 0,
             query: 'SELECT * FROM "CarCache".Car',
             result: 'table',
             timeLineSpan: '1',
@@ -49,8 +51,9 @@ const DEMO_NOTEBOOK = {
         },
         {
             name: 'Query with aggregates',
-            cacheName: 'CarCache',
-            pageSize: 50,
+            cacheName: 'ParkingCache',
+            pageSize: 100,
+            limit: 0,
             query: [
                 'SELECT p.name, count(*) AS cnt',
                 'FROM "ParkingCache".Parking p',

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/sql/Notebook.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/sql/Notebook.service.js b/modules/web-console/frontend/app/modules/sql/Notebook.service.js
index 12730be..f32d26f 100644
--- a/modules/web-console/frontend/app/modules/sql/Notebook.service.js
+++ b/modules/web-console/frontend/app/modules/sql/Notebook.service.js
@@ -60,7 +60,7 @@ export default class Notebook {
     }
 
     remove(notebook) {
-        return this.confirmModal.confirm(`Are you sure you want to remove: "${notebook.name}"?`)
+        return this.confirmModal.confirm(`Are you sure you want to remove notebook: "${notebook.name}"?`)
             .then(() => this.NotebookData.findIndex(notebook))
             .then((idx) => {
                 this.NotebookData.remove(notebook)

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade b/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade
deleted file mode 100644
index 0396727..0000000
--- a/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade
+++ /dev/null
@@ -1,39 +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.
-
-include /app/helpers/jade/mixins.jade
-
-.modal(tabindex='-1' role='dialog')
-    .modal-dialog
-        .modal-content
-            .modal-header
-                button.close(ng-click='$hide()') &times;
-                h4.modal-title Scan filter
-            form.form-horizontal.modal-body.row(name='ui.inputForm' novalidate)
-                .settings-row
-                    .col-sm-2
-                        label.required.labelFormField Filter:&nbsp;
-                    .col-sm-10
-                        .input-tip
-                            +ignite-form-field-input('"filter"', 'ui.filter', false, 'true', 'Enter filter')(
-                                data-ignite-form-field-input-autofocus='true'
-                                ignite-on-enter='form.$valid && ok()'
-                            )
-                .settings-row
-                    +checkbox('Case sensitive search', 'ui.caseSensitive', '"caseSensitive"', 'Select this checkbox for case sensitive search')
-            .modal-footer
-                button.btn.btn-default(id='btn-cancel' ng-click='$hide()') Cancel
-                button.btn.btn-primary(id='btn-scan' ng-disabled='ui.inputForm.$invalid' ng-click='ok()') Scan

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/sql/scan-filter-input.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/sql/scan-filter-input.service.js b/modules/web-console/frontend/app/modules/sql/scan-filter-input.service.js
deleted file mode 100644
index 18ba3ba..0000000
--- a/modules/web-console/frontend/app/modules/sql/scan-filter-input.service.js
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-export default class ScanFilter {
-    static $inject = ['$rootScope', '$q', '$modal'];
-
-    constructor($root, $q, $modal) {
-        this.deferred = null;
-        this.$q = $q;
-
-        const scope = $root.$new();
-
-        scope.ui = {};
-
-        scope.ok = () => {
-            this.deferred.resolve({filter: scope.ui.filter, caseSensitive: !!scope.ui.caseSensitive});
-
-            this.modal.hide();
-        };
-
-        scope.$hide = () => {
-            this.modal.hide();
-
-            this.deferred.reject();
-        };
-
-        this.modal = $modal({templateUrl: '/scan-filter-input.html', scope, placement: 'center', show: false});
-    }
-
-    open() {
-        this.deferred = this.$q.defer();
-
-        this.modal.$promise.then(this.modal.show);
-
-        return this.deferred.promise;
-    }
-}


[16/50] [abbrv] ignite git commit: IGNITE-3886 .NET: Build script added

Posted by yz...@apache.org.
IGNITE-3886 .NET: Build script added

This closes #1298


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

Branch: refs/heads/ignite-comm-balance-master
Commit: c864fe443284508c7eb1d2341d9fdc2a5a844c45
Parents: 0a33644
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Wed Dec 21 15:50:36 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Wed Dec 21 15:50:36 2016 +0300

----------------------------------------------------------------------
 modules/platforms/dotnet/Apache.Ignite.sln |   2 +
 modules/platforms/dotnet/DEVNOTES.txt      |  12 +-
 modules/platforms/dotnet/build.bat         |  14 ++
 modules/platforms/dotnet/build.ps1         | 211 ++++++++++++++++++++++++
 4 files changed, 236 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c864fe44/modules/platforms/dotnet/Apache.Ignite.sln
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.sln b/modules/platforms/dotnet/Apache.Ignite.sln
index ef953b1..bcaa753 100644
--- a/modules/platforms/dotnet/Apache.Ignite.sln
+++ b/modules/platforms/dotnet/Apache.Ignite.sln
@@ -30,6 +30,8 @@ Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "Solution Items", "Solution
 		Apache.Ignite.FxCop = Apache.Ignite.FxCop
 		Apache.Ignite.sln.DotSettings = Apache.Ignite.sln.DotSettings
 		Apache.Ignite.sln.TeamCity.DotSettings = Apache.Ignite.sln.TeamCity.DotSettings
+		build.bat = build.bat
+		build.ps1 = build.ps1
 		DEVNOTES.txt = DEVNOTES.txt
 		README.txt = README.txt
 		README.md = README.md

http://git-wip-us.apache.org/repos/asf/ignite/blob/c864fe44/modules/platforms/dotnet/DEVNOTES.txt
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/DEVNOTES.txt b/modules/platforms/dotnet/DEVNOTES.txt
index 630960e..df0ebb5 100644
--- a/modules/platforms/dotnet/DEVNOTES.txt
+++ b/modules/platforms/dotnet/DEVNOTES.txt
@@ -5,12 +5,18 @@ Requirements:
 * Windows (XP and up), Windows Server (2008 and up)
 * Oracle JDK 7 and above
 * .NET Framework 4.0
+* PowerShell 3.0+
 * Visual Studio 2010 (later versions require upgrading "common" C++ project, see below)
-* JAVA_HOME environment variable set to the corresponding JDK (x64 or x86).
+* JAVA_HOME environment variable set to the corresponding JDK (x64 or x86)
+* Apache Maven bin directory in PATH, or MAVEN_HOME environment variable
 
 Building binaries:
-  msbuild Apache.Ignite.sln /p:Configuration=Release
-Resulting binaries will be in Apache.Ignite\bin folder
+  build.bat -skipCodeAnalysis
+Resulting binaries will be in bin folder, and NuGet packages in nupkg folder.
+
+Running built binaries: resulting "bin" folder in self contained, you can copy it anywhere and run
+  bin\Apache.Ignite.exe  
+
 NOTE: 
 * x86 solution platform requires x86 Oracle JDK.
 * x64 solution platform requires x64 Oracle JDK.

http://git-wip-us.apache.org/repos/asf/ignite/blob/c864fe44/modules/platforms/dotnet/build.bat
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/build.bat b/modules/platforms/dotnet/build.bat
new file mode 100644
index 0000000..ef0a96f
--- /dev/null
+++ b/modules/platforms/dotnet/build.bat
@@ -0,0 +1,14 @@
+:: Licensed 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.
+
+:: Apache Ignite.NET build script runner, see build.ps1 for more details.
+powershell -executionpolicy remotesigned -file build.ps1 %*
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c864fe44/modules/platforms/dotnet/build.ps1
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/build.ps1 b/modules/platforms/dotnet/build.ps1
new file mode 100644
index 0000000..be7e638
--- /dev/null
+++ b/modules/platforms/dotnet/build.ps1
@@ -0,0 +1,211 @@
+<#
+
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+
+ #>
+
+ <#
+
+.SYNOPSIS
+Apache Ignite.NET build script.
+
+.DESCRIPTION
+Builds all parts of Apache Ignite.NET: Java, .NET, NuGet. Copies results to 'bin' and 'nupkg' folders.
+
+Requirements:
+* PowerShell 3
+* JDK 7+
+* MAVEN_HOME environment variable or mvn.bat in PATH
+
+.PARAMETER skipJava
+Skip Java build.
+
+.PARAMETER skipNuGet
+Skip NuGet packaging.
+
+.PARAMETER skipCodeAnalysis
+Skip code analysis.
+
+.PARAMETER clean
+Perform a clean rebuild.
+
+.PARAMETER platform
+Build platform ("Any CPU", "x86", "x64").
+
+.PARAMETER configuration
+Build configuration ("Release", "Debug").
+
+.PARAMETER mavenOpts
+Custom Maven options, default is "-U -P-lgpl,-scala,-examples,-test,-benchmarks -Dmaven.javadoc.skip=true".
+
+.EXAMPLE
+.\build.ps1 -clean  
+# Full rebuild of Java, .NET and NuGet packages.
+
+.EXAMPLE
+.\build.ps1 -skipJava -skipCodeAnalysis -skipNuGet -configuration Debug -platform x64
+# Quick build of .NET code only.
+
+#>
+
+param (
+    [switch]$skipJava,
+    [switch]$skipNuGet,
+    [switch]$skipCodeAnalysis,  
+    [switch]$clean,
+    [ValidateSet("Any CPU", "x64", "x86")]
+    [string]$platform="Any CPU",
+    [ValidateSet("Release", "Debug")]
+    [string]$configuration="Release",
+    [string]$mavenOpts="-U -P-lgpl,-scala,-examples,-test,-benchmarks -Dmaven.javadoc.skip=true"
+ )
+
+# 1) Build Java (Maven)
+if (!$skipJava) {
+    # Detect Ignite root directory
+    cd $PSScriptRoot\..
+
+    while (!((Test-Path bin) -and (Test-Path examples) -and ((Test-Path modules) -or (Test-Path platforms))))
+    { cd .. }
+
+    echo "Ignite home detected at '$pwd'."
+
+    # Detect Maven
+    $mv = "mvn"
+    if ((Get-Command $mv -ErrorAction SilentlyContinue) -eq $null) { 
+        $mvHome = ($env:MAVEN_HOME, $env:M2_HOME, $env:M3_HOME, $env:MVN_HOME -ne $null)[0]
+
+        if ($mvHome -eq $null) {
+            echo "Maven not found. Make sure to update PATH variable or set MAVEN_HOME, M2_HOME, M3_HOME, or MVN_HOME."
+            exit -1
+        }
+
+        $mv = "`"" + (join-path $mvHome "bin\mvn.bat") + "`""
+        echo "Maven detected at $mv."
+    }
+
+    # Run Maven
+    echo "Starting Java (Maven) build..."
+    
+    $mvnTargets = if ($clean)  { "clean package" } else { "package" }
+    cmd /c "$mv $mvnTargets -DskipTests $mavenOpts"
+
+    # Check result
+    if ($LastExitCode -ne 0) {
+        echo "Java (Maven) build failed."; exit -1
+    }
+
+    # Copy (relevant) jars
+    $libsDir = "$PSScriptRoot\bin\Libs"
+    mkdir -Force $libsDir; del -Force $libsDir\*.*
+    
+    copy -Force target\release-package\libs\*.jar $libsDir
+    copy -Force target\release-package\libs\ignite-spring\*.jar $libsDir
+    copy -Force target\release-package\libs\ignite-indexing\*.jar $libsDir
+    copy -Force target\release-package\libs\licenses\*.jar $libsDir
+
+    # Restore directory
+    cd $PSScriptRoot
+}
+else {
+    echo "Java (Maven) build skipped."
+}
+
+
+# 2) Build .NET
+# Detect MSBuild 4.0+
+for ($i=4; $i -le 20; $i++) {
+    $regKey = "HKLM:\software\Microsoft\MSBuild\ToolsVersions\$i.0"
+    if (Test-Path $regKey) { break }
+}
+
+if (!(Test-Path $regKey)) {
+    echo "Failed to detect MSBuild path, exiting."
+    exit -1
+}
+
+$msbuildExe = (join-path -path (Get-ItemProperty $regKey)."MSBuildToolsPath" -childpath "msbuild.exe")
+echo "MSBuild detected at '$msbuildExe'."
+
+# Detect NuGet
+$ng = "nuget"
+if ((Get-Command $ng -ErrorAction SilentlyContinue) -eq $null) { 
+    echo "Downloading NuGet..."
+    (New-Object System.Net.WebClient).DownloadFile("https://dist.nuget.org/win-x86-commandline/v3.3.0/nuget.exe", "nuget.exe");    
+    $ng = ".\nuget.exe"
+}
+
+# Restore NuGet packages
+echo "Restoring NuGet..."
+& $ng restore
+
+# Build
+echo "Starting MsBuild..."
+$targets = if ($clean) {"Clean;Rebuild"} else {"Build"}
+$codeAnalysis = if ($skipCodeAnalysis) {"/p:RunCodeAnalysis=false"} else {""}
+& $msbuildExe Apache.Ignite.sln /target:$targets /p:Configuration=$configuration /p:Platform=`"$platform`" $codeAnalysis /p:UseSharedCompilation=false
+
+# Check result
+if ($LastExitCode -ne 0) {
+    echo ".NET build failed."
+    exit -1
+}
+
+# Copy binaries
+mkdir -Force bin; del -Force bin\*.*
+
+ls *.csproj -Recurse | where Name -NotLike "*Examples*" `
+                     | where Name -NotLike "*Tests*" `
+                     | where Name -NotLike "*Benchmarks*" | % {
+    $binDir = if (($configuration -eq "Any CPU") -or ($_.Name -ne "Apache.Ignite.Core.csproj")) `
+                {"bin\$configuration"} else {"bin\$platform\$configuration"}
+    $dir = join-path (split-path -parent $_) $binDir    
+    xcopy /s /y $dir\*.* bin
+}
+
+
+# 3) Pack NuGet
+if (!$skipNuGet) {
+    # Check parameters
+    if (($platform -ne "Any CPU") -or ($configuration -ne "Release")) {
+        echo "NuGet can only package 'Release' 'Any CPU' builds; you have specified '$configuration' '$platform'."
+        exit -1
+    }
+
+    $nupkgDir = "nupkg"
+    mkdir -Force $nupkgDir; del -Force $nupkgDir\*.*
+
+    # Detect version
+    $ver = (gi Apache.Ignite.Core\bin\Release\Apache.Ignite.Core.dll).VersionInfo.ProductVersion
+
+    # Find all nuspec files and run 'nuget pack' either directly, or on corresponding csproj files (if present)
+    ls *.nuspec -Recurse  `
+        | % { 
+            If (Test-Path ([io.path]::ChangeExtension($_.FullName, ".csproj"))){
+                [io.path]::ChangeExtension($_.FullName, ".csproj")
+            } Else { $_.FullName }
+        } | % { 
+            & $ng pack $_ -Prop Configuration=Release -Prop Platform=AnyCPU -Version $ver -OutputDirectory $nupkgDir
+
+            # check result
+            if ($LastExitCode -ne 0)
+            {
+                echo "NuGet pack failed."; exit -1
+            }
+        }
+
+    echo "NuGet packages created in '$pwd\$nupkgDir'."
+}
\ No newline at end of file


[37/50] [abbrv] ignite git commit: Implemented Visor tasks for Services.

Posted by yz...@apache.org.
Implemented Visor tasks for Services.


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

Branch: refs/heads/ignite-comm-balance-master
Commit: fdf1f4bb038eb9b0918367c7a5b7e49fb1e9ec51
Parents: 8cffe90
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Dec 23 18:20:44 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Dec 23 18:20:44 2016 +0700

----------------------------------------------------------------------
 .../visor/service/VisorCancelServiceTask.java   |  70 ++++++++++
 .../visor/service/VisorServiceDescriptor.java   | 132 +++++++++++++++++++
 .../visor/service/VisorServiceTask.java         |  75 +++++++++++
 .../internal/visor/util/VisorTaskUtils.java     |  15 ++-
 .../resources/META-INF/classnames.properties    |  65 +++++++--
 5 files changed, 342 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fdf1f4bb/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorCancelServiceTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorCancelServiceTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorCancelServiceTask.java
new file mode 100644
index 0000000..64987e9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorCancelServiceTask.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.service;
+
+import org.apache.ignite.IgniteServices;
+import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.visor.VisorJob;
+import org.apache.ignite.internal.visor.VisorOneNodeTask;
+
+/**
+ * Task for cancel services with specified name.
+ */
+@GridInternal
+public class VisorCancelServiceTask extends VisorOneNodeTask<String, Void> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override protected VisorCancelServiceJob job(String arg) {
+        return new VisorCancelServiceJob(arg, debug);
+    }
+
+    /**
+     * Job for cancel services with specified name.
+     */
+    private static class VisorCancelServiceJob extends VisorJob<String, Void> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         * Create job with specified argument.
+         *
+         * @param arg Job argument.
+         * @param debug Debug flag.
+         */
+        protected VisorCancelServiceJob(String arg, boolean debug) {
+            super(arg, debug);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected Void run(final String arg) {
+            IgniteServices services = ignite.services();
+
+            services.cancel(arg);
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(VisorCancelServiceJob.class, this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdf1f4bb/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.java
new file mode 100644
index 0000000..83ec77d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.java
@@ -0,0 +1,132 @@
+/*
+ * 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.visor.service;
+
+import java.io.Serializable;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.visor.util.VisorTaskUtils;
+import org.apache.ignite.services.ServiceDescriptor;
+
+/**
+ * Data transfer object for {@link ServiceDescriptor} object.
+ */
+public class VisorServiceDescriptor implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Service name. */
+    private String name;
+
+    /** Service class. */
+    private String srvcCls;
+
+    /** Maximum allowed total number of deployed services in the grid, {@code 0} for unlimited. */
+    private int totalCnt;
+
+    /** Maximum allowed number of deployed services on each node. */
+    private int maxPerNodeCnt;
+
+    /** Cache name used for key-to-node affinity calculation. */
+    private String cacheName;
+
+    /** ID of grid node that initiated the service deployment. */
+    private UUID originNodeId;
+
+    /**
+     * Service deployment topology snapshot.
+     * Number of service instances deployed on a node mapped to node ID.
+     */
+    private Map<UUID, Integer> topSnapshot;
+
+    /**
+     * Default constructor.
+     */
+    public VisorServiceDescriptor() {
+        // No-op.
+    }
+
+    /**
+     * Create task result with given parameters
+     *
+     */
+    public VisorServiceDescriptor(ServiceDescriptor srvc) {
+        name = srvc.name();
+        srvcCls = VisorTaskUtils.compactClass(srvc.serviceClass());
+        totalCnt = srvc.totalCount();
+        maxPerNodeCnt = srvc.maxPerNodeCount();
+        cacheName = srvc.cacheName();
+        originNodeId = srvc.originNodeId();
+        topSnapshot = srvc.topologySnapshot();
+    }
+
+    /**
+     * @return Service name.
+     */
+    public String getName() {
+        return name;
+    }
+
+    /**
+     * @return Service class.
+     */
+    public String getServiceClass() {
+        return srvcCls;
+    }
+
+    /**
+     * @return Maximum allowed total number of deployed services in the grid, 0 for unlimited.
+     */
+    public int getTotalCnt() {
+        return totalCnt;
+    }
+
+    /**
+     * @return Maximum allowed number of deployed services on each node.
+     */
+    public int getMaxPerNodeCnt() {
+        return maxPerNodeCnt;
+    }
+
+    /**
+     * @return Cache name used for key-to-node affinity calculation.
+     */
+    public String getCacheName() {
+        return cacheName;
+    }
+
+    /**
+     * @return ID of grid node that initiated the service deployment.
+     */
+    public UUID getOriginNodeId() {
+        return originNodeId;
+    }
+
+    /**
+     * @return Service deployment topology snapshot. Number of service instances deployed on a node mapped to node ID.
+     */
+    public Map<UUID, Integer> getTopologySnapshot() {
+        return topSnapshot;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorServiceDescriptor.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdf1f4bb/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java
new file mode 100644
index 0000000..1b3495c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.service;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.visor.VisorJob;
+import org.apache.ignite.internal.visor.VisorOneNodeTask;
+import org.apache.ignite.services.ServiceDescriptor;
+
+/**
+ * Task for collect topology service configuration.
+ */
+@GridInternal
+public class VisorServiceTask extends VisorOneNodeTask<Void, Collection<VisorServiceDescriptor>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override protected VisorServiceJob job(Void arg) {
+        return new VisorServiceJob(arg, debug);
+    }
+
+    /**
+     * Job for collect topology service configuration.
+     */
+    private static class VisorServiceJob extends VisorJob<Void, Collection<VisorServiceDescriptor>> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         * Create job with specified argument.
+         *
+         * @param arg Job argument.
+         * @param debug Debug flag.
+         */
+        protected VisorServiceJob(Void arg, boolean debug) {
+            super(arg, debug);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected Collection<VisorServiceDescriptor> run(final Void arg) {
+            Collection<ServiceDescriptor> services = ignite.services().serviceDescriptors();
+
+            Collection<VisorServiceDescriptor> res = new ArrayList<>(services.size());
+
+            for (ServiceDescriptor srvc: services)
+                res.add(new VisorServiceDescriptor(srvc));
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(VisorServiceJob.class, this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdf1f4bb/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
index 1e9346c..3f5003a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
@@ -270,6 +270,19 @@ public class VisorTaskUtils {
     /**
      * Compact class names.
      *
+     * @param cls Class object for compact.
+     * @return Compacted string.
+     */
+    @Nullable public static String compactClass(Class cls) {
+        if (cls == null)
+            return null;
+
+        return U.compact(cls.getName());
+    }
+
+    /**
+     * Compact class names.
+     *
      * @param obj Object for compact.
      * @return Compacted string.
      */
@@ -277,7 +290,7 @@ public class VisorTaskUtils {
         if (obj == null)
             return null;
 
-        return U.compact(obj.getClass().getName());
+        return compactClass(obj.getClass());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdf1f4bb/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index 4c9596c..4d0b931 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -294,12 +294,17 @@ org.apache.ignite.internal.jdbc2.JdbcDatabaseMetadata$UpdateMetadataTask
 org.apache.ignite.internal.jdbc2.JdbcQueryTask
 org.apache.ignite.internal.jdbc2.JdbcQueryTask$1
 org.apache.ignite.internal.jdbc2.JdbcQueryTask$QueryResult
+org.apache.ignite.internal.jdbc2.JdbcQueryTaskV2
+org.apache.ignite.internal.jdbc2.JdbcQueryTaskV2$1
+org.apache.ignite.internal.jdbc2.JdbcQueryTaskV2$QueryResult
+org.apache.ignite.internal.jdbc2.JdbcSqlFieldsQuery
 org.apache.ignite.internal.managers.GridManagerAdapter$1$1
 org.apache.ignite.internal.managers.checkpoint.GridCheckpointManager$CheckpointSet
 org.apache.ignite.internal.managers.checkpoint.GridCheckpointRequest
 org.apache.ignite.internal.managers.communication.GridIoManager$ConcurrentHashMap0
 org.apache.ignite.internal.managers.communication.GridIoMessage
 org.apache.ignite.internal.managers.communication.GridIoUserMessage
+org.apache.ignite.internal.managers.communication.IgniteIoTestMessage
 org.apache.ignite.internal.managers.deployment.GridDeploymentInfoBean
 org.apache.ignite.internal.managers.deployment.GridDeploymentPerVersionStore$2
 org.apache.ignite.internal.managers.deployment.GridDeploymentRequest
@@ -387,20 +392,20 @@ org.apache.ignite.internal.processors.cache.GridCacheAdapter$3
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$30
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$32
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$4
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$48
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$49
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$50
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$51
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$52
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$53
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$54
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$55
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$57
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$58
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$58$1
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$59
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$6
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$60
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$61
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$62
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$63
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$64
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$65
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$66
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$67
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$69
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$70
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$70$1
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$71
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$72
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$9
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$AsyncOp$1
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$AsyncOp$1$1
@@ -719,8 +724,11 @@ org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtFor
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponse
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$1
+org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$1$1
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$2
+org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$3
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$4$1
+org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$5$1
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$DemandWorker$1
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$DemandWorker$2
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId
@@ -1110,6 +1118,12 @@ org.apache.ignite.internal.processors.hadoop.HadoopJobStatus
 org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan
 org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo
 org.apache.ignite.internal.processors.hadoop.HadoopTaskType
+org.apache.ignite.internal.processors.hadoop.message.HadoopMessage
+org.apache.ignite.internal.processors.hadoop.shuffle.HadoopDirectShuffleMessage
+org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleAck
+org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleFinishRequest
+org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleFinishResponse
+org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleMessage
 org.apache.ignite.internal.processors.igfs.IgfsAckMessage
 org.apache.ignite.internal.processors.igfs.IgfsAttributes
 org.apache.ignite.internal.processors.igfs.IgfsBlockKey
@@ -1207,6 +1221,7 @@ org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryProcessor
 org.apache.ignite.internal.processors.platform.cache.PlatformCachePartialUpdateException
 org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinity$1
 org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinityFunction
+org.apache.ignite.internal.processors.platform.cache.expiry.PlatformExpiryPolicyFactory
 org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQuery
 org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryFilter
 org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryImpl
@@ -1243,6 +1258,9 @@ org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$9
 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetConfigurationClosure
 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetService
 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetServiceImpl
+org.apache.ignite.internal.processors.platform.entityframework.PlatformDotNetEntityFrameworkCacheExtension$CleanupCompletionListener
+org.apache.ignite.internal.processors.platform.entityframework.PlatformDotNetEntityFrameworkCacheExtension$RemoveOldEntriesRunnable
+org.apache.ignite.internal.processors.platform.entityframework.PlatformDotNetEntityFrameworkIncreaseVersionProcessor
 org.apache.ignite.internal.processors.platform.events.PlatformEventFilterListenerImpl
 org.apache.ignite.internal.processors.platform.message.PlatformMessageFilter
 org.apache.ignite.internal.processors.platform.messaging.PlatformMessageFilterImpl
@@ -1265,6 +1283,7 @@ org.apache.ignite.internal.processors.query.GridQueryProcessor$6
 org.apache.ignite.internal.processors.query.GridQueryProcessor$7
 org.apache.ignite.internal.processors.query.GridQueryProcessor$8
 org.apache.ignite.internal.processors.query.GridQueryProcessor$IndexType
+org.apache.ignite.internal.processors.query.IgniteSQLException
 org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest
 org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse
 org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest
@@ -1330,6 +1349,9 @@ org.apache.ignite.internal.processors.rest.handlers.datastructures.DataStructure
 org.apache.ignite.internal.processors.rest.handlers.query.CacheQueryFieldsMetaResult
 org.apache.ignite.internal.processors.rest.handlers.query.CacheQueryResult
 org.apache.ignite.internal.processors.rest.handlers.query.QueryCommandHandler$QueryCursorIterator
+org.apache.ignite.internal.processors.rest.handlers.redis.GridRedisRestCommandHandler$1
+org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisGenericException
+org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisTypeException
 org.apache.ignite.internal.processors.rest.handlers.task.GridTaskCommandHandler$2
 org.apache.ignite.internal.processors.rest.handlers.task.GridTaskCommandHandler$ExeCallable
 org.apache.ignite.internal.processors.rest.handlers.task.GridTaskResultRequest
@@ -1341,6 +1363,9 @@ org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpMemcachedNioList
 org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpMemcachedNioListener$2
 org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpRestNioListener$1
 org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpRestNioListener$1$1
+org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand
+org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisMessage
+org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisNioListener$1
 org.apache.ignite.internal.processors.rest.request.RestQueryRequest$QueryType
 org.apache.ignite.internal.processors.service.GridServiceAssignments
 org.apache.ignite.internal.processors.service.GridServiceAssignmentsKey
@@ -1585,10 +1610,13 @@ org.apache.ignite.internal.util.lang.IgniteReducer2X
 org.apache.ignite.internal.util.lang.IgniteReducer3
 org.apache.ignite.internal.util.lang.IgniteReducer3X
 org.apache.ignite.internal.util.lang.IgniteReducerX
+org.apache.ignite.internal.util.lang.IgniteSingletonIterator
 org.apache.ignite.internal.util.nio.GridNioEmbeddedFuture$1
 org.apache.ignite.internal.util.nio.GridNioException
 org.apache.ignite.internal.util.nio.GridNioMessageTracker
 org.apache.ignite.internal.util.nio.GridNioServer$NioOperation
+org.apache.ignite.internal.util.nio.GridNioServer$RandomBalancer
+org.apache.ignite.internal.util.nio.GridNioServer$SizeBasedBalancer
 org.apache.ignite.internal.util.nio.GridNioSessionMetaKey
 org.apache.ignite.internal.util.nio.ssl.GridNioSslHandler
 org.apache.ignite.internal.util.offheap.GridOffHeapEvent
@@ -1801,6 +1829,11 @@ org.apache.ignite.internal.visor.query.VisorQueryResult
 org.apache.ignite.internal.visor.query.VisorQueryResultEx
 org.apache.ignite.internal.visor.query.VisorQueryScanSubstringFilter
 org.apache.ignite.internal.visor.query.VisorQueryTask
+org.apache.ignite.internal.visor.service.VisorCancelServiceTask
+org.apache.ignite.internal.visor.service.VisorCancelServiceTask$VisorCancelServiceJob
+org.apache.ignite.internal.visor.service.VisorServiceDescriptor
+org.apache.ignite.internal.visor.service.VisorServiceTask
+org.apache.ignite.internal.visor.service.VisorServiceTask$VisorServiceJob
 org.apache.ignite.internal.visor.util.VisorClusterGroupEmptyException
 org.apache.ignite.internal.visor.util.VisorEventMapper
 org.apache.ignite.internal.visor.util.VisorExceptionWrapper
@@ -1858,12 +1891,15 @@ org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointData
 org.apache.ignite.spi.collision.jobstealing.JobStealingRequest
 org.apache.ignite.spi.collision.priorityqueue.PriorityQueueCollisionSpi$PriorityGridCollisionJobContextComparator
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$1
+org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$10
+org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$11
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosure
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosure$1
-org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$8
-org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$9
+org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosureNew
+org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosureNew$1
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeClosure
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeMessage
+org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeMessage2
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeTimeoutException
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$NodeIdMessage
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$RecoveryLastReceivedMessage
@@ -1923,3 +1959,4 @@ org.apache.ignite.transactions.TransactionOptimisticException
 org.apache.ignite.transactions.TransactionRollbackException
 org.apache.ignite.transactions.TransactionState
 org.apache.ignite.transactions.TransactionTimeoutException
+org.apache.ignite.util.AttributeNodeFilter


[48/50] [abbrv] ignite git commit: IGNITE-3430 .NET: Refactor transactional tests to a separate class

Posted by yz...@apache.org.
IGNITE-3430 .NET: Refactor transactional tests to a separate class


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 864af7eb48f19b8d6350332100f741fcdccdb5ad
Parents: 06ef846
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Wed Dec 28 13:55:26 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Wed Dec 28 13:55:26 2016 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.Tests.csproj             |   1 +
 .../Cache/CacheAbstractTest.cs                  | 625 -------------------
 .../Cache/CacheAbstractTransactionalTest.cs     | 556 +++++++++++++++++
 .../Cache/CacheLocalAtomicTest.cs               |   5 -
 .../Cache/CacheLocalTest.cs                     |   6 +-
 .../CachePartitionedAtomicNearEnabledTest.cs    |   5 -
 .../Cache/CachePartitionedAtomicTest.cs         |   5 -
 .../Cache/CachePartitionedNearEnabledTest.cs    |   7 +-
 .../Cache/CachePartitionedTest.cs               |   7 +-
 .../Cache/CacheReplicatedAtomicTest.cs          |   5 -
 .../Cache/CacheReplicatedTest.cs                |   7 +-
 11 files changed, 561 insertions(+), 668 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/864af7eb/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 f440c25..5948593 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
@@ -71,6 +71,7 @@
     <Compile Include="Cache\CacheMetricsTest.cs" />
     <Compile Include="Cache\CacheResultTest.cs" />
     <Compile Include="Cache\CacheSwapSpaceTest.cs" />
+    <Compile Include="Cache\CacheAbstractTransactionalTest.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/864af7eb/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 821a179..250f974 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
@@ -1917,621 +1917,6 @@ namespace Apache.Ignite.Core.Tests.Cache
         }
 
         /// <summary>
-        /// Simple cache lock test (while <see cref="TestLock"/> is ignored).
-        /// </summary>
-        [Test]
-        public void TestLockSimple()
-        {
-            if (!LockingEnabled())
-                return;
-
-            var cache = Cache();
-
-            const int key = 7;
-
-            Action<ICacheLock> checkLock = lck =>
-            {
-                using (lck)
-                {
-                    Assert.Throws<InvalidOperationException>(lck.Exit); // can't exit if not entered
-
-                    lck.Enter();
-
-                    Assert.IsTrue(cache.IsLocalLocked(key, true));
-                    Assert.IsTrue(cache.IsLocalLocked(key, false));
-
-                    lck.Exit();
-
-                    Assert.IsFalse(cache.IsLocalLocked(key, true));
-                    Assert.IsFalse(cache.IsLocalLocked(key, false));
-
-                    Assert.IsTrue(lck.TryEnter());
-
-                    Assert.IsTrue(cache.IsLocalLocked(key, true));
-                    Assert.IsTrue(cache.IsLocalLocked(key, false));
-
-                    lck.Exit();
-                }
-
-                Assert.Throws<ObjectDisposedException>(lck.Enter); // Can't enter disposed lock
-            };
-
-            checkLock(cache.Lock(key));
-            checkLock(cache.LockAll(new[] {key, 1, 2, 3}));
-        }
-
-        [Test]
-        [Ignore("IGNITE-835")]
-        public void TestLock()
-        {
-            if (!LockingEnabled())
-                return;
-
-            var cache = Cache();
-
-            const int key = 7;
-
-            // Lock
-            CheckLock(cache, key, () => cache.Lock(key));
-
-            // LockAll
-            CheckLock(cache, key, () => cache.LockAll(new[] { key, 2, 3, 4, 5 }));
-        }
-
-        /// <summary>
-        /// Internal lock test routine.
-        /// </summary>
-        /// <param name="cache">Cache.</param>
-        /// <param name="key">Key.</param>
-        /// <param name="getLock">Function to get the lock.</param>
-        private static void CheckLock(ICache<int, int> cache, int key, Func<ICacheLock> getLock)
-        {
-            var sharedLock = getLock();
-            
-            using (sharedLock)
-            {
-                Assert.Throws<InvalidOperationException>(() => sharedLock.Exit());  // can't exit if not entered
-
-                sharedLock.Enter();
-
-                try
-                {
-                    Assert.IsTrue(cache.IsLocalLocked(key, true));
-                    Assert.IsTrue(cache.IsLocalLocked(key, false));
-
-                    EnsureCannotLock(getLock, sharedLock);
-
-                    sharedLock.Enter();
-
-                    try
-                    {
-                        Assert.IsTrue(cache.IsLocalLocked(key, true));
-                        Assert.IsTrue(cache.IsLocalLocked(key, false));
-
-                        EnsureCannotLock(getLock, sharedLock);
-                    }
-                    finally
-                    {
-                        sharedLock.Exit();
-                    }
-
-                    Assert.IsTrue(cache.IsLocalLocked(key, true));
-                    Assert.IsTrue(cache.IsLocalLocked(key, false));
-
-                    EnsureCannotLock(getLock, sharedLock);
-
-                    Assert.Throws<SynchronizationLockException>(() => sharedLock.Dispose()); // can't dispose while locked
-                }
-                finally
-                {
-                    sharedLock.Exit();
-                }
-
-                Assert.IsFalse(cache.IsLocalLocked(key, true));
-                Assert.IsFalse(cache.IsLocalLocked(key, false));
-
-                var innerTask = new Task(() =>
-                {
-                    Assert.IsTrue(sharedLock.TryEnter());
-                    sharedLock.Exit();
-
-                    using (var otherLock = getLock())
-                    {
-                        Assert.IsTrue(otherLock.TryEnter());
-                        otherLock.Exit();
-                    }
-                });
-
-                innerTask.Start();
-                innerTask.Wait();
-            }
-            
-            Assert.IsFalse(cache.IsLocalLocked(key, true));
-            Assert.IsFalse(cache.IsLocalLocked(key, false));
-            
-            var outerTask = new Task(() =>
-            {
-                using (var otherLock = getLock())
-                {
-                    Assert.IsTrue(otherLock.TryEnter());
-                    otherLock.Exit();
-                }
-            });
-
-            outerTask.Start();
-            outerTask.Wait();
-
-            Assert.Throws<ObjectDisposedException>(() => sharedLock.Enter());  // Can't enter disposed lock
-        }
-
-        /// <summary>
-        /// Ensure that lock cannot be obtained by other threads.
-        /// </summary>
-        /// <param name="getLock">Get lock function.</param>
-        /// <param name="sharedLock">Shared lock.</param>
-        private static void EnsureCannotLock(Func<ICacheLock> getLock, ICacheLock sharedLock)
-        {
-            var task = new Task(() =>
-            {
-                Assert.IsFalse(sharedLock.TryEnter());
-                Assert.IsFalse(sharedLock.TryEnter(TimeSpan.FromMilliseconds(100)));
-
-                using (var otherLock = getLock())
-                {
-                    Assert.IsFalse(otherLock.TryEnter());
-                    Assert.IsFalse(otherLock.TryEnter(TimeSpan.FromMilliseconds(100)));
-                }
-            });
-
-            task.Start();
-            task.Wait();
-        }
-
-        [Test]
-        public void TestTxCommit()
-        {
-            TestTxCommit(false);
-        }
-
-        [Test]
-        public void TestTxCommitAsync()
-        {
-            TestTxCommit(true);
-        }
-
-        private void TestTxCommit(bool async)
-        {
-            if (!TxEnabled())
-                return;
-
-            var cache = Cache();
-
-            ITransaction tx = Transactions.Tx;
-
-            Assert.IsNull(tx);
-
-            tx = Transactions.TxStart();
-
-            try
-            {
-                cache.Put(1, 1);
-
-                cache.Put(2, 2);
-
-                if (async)
-                {
-                    var task = tx.CommitAsync();
-
-                    task.Wait();
-
-                    Assert.IsTrue(task.IsCompleted);
-                }
-                else
-                    tx.Commit();
-            }
-            finally
-            {
-                tx.Dispose();
-            }
-
-            Assert.AreEqual(1, cache.Get(1));
-
-            Assert.AreEqual(2, cache.Get(2));
-
-            tx = Transactions.Tx;
-
-            Assert.IsNull(tx);
-        }
-
-        [Test]
-        public void TestTxRollback()
-        {
-            if (!TxEnabled())
-                return;
-
-            var cache = Cache();
-
-            cache.Put(1, 1);
-
-            cache.Put(2, 2);
-
-            ITransaction tx = Transactions.Tx;
-
-            Assert.IsNull(tx);
-
-            tx = Transactions.TxStart();
-
-            try {
-                cache.Put(1, 10);
-
-                cache.Put(2, 20);
-            }
-            finally {
-                tx.Rollback();
-            }
-
-            Assert.AreEqual(1, cache.Get(1));
-
-            Assert.AreEqual(2, cache.Get(2));
-
-            Assert.IsNull(Transactions.Tx);
-        }
-
-        [Test]
-        public void TestTxClose()
-        {
-            if (!TxEnabled())
-                return;
-
-            var cache = Cache();
-
-            cache.Put(1, 1);
-
-            cache.Put(2, 2);
-
-            ITransaction tx = Transactions.Tx;
-
-            Assert.IsNull(tx);
-
-            tx = Transactions.TxStart();
-
-            try
-            {
-                cache.Put(1, 10);
-
-                cache.Put(2, 20);
-            }
-            finally
-            {
-                tx.Dispose();
-            }
-
-            Assert.AreEqual(1, cache.Get(1));
-
-            Assert.AreEqual(2, cache.Get(2));
-
-            tx = Transactions.Tx;
-
-            Assert.IsNull(tx);
-        }
-        
-        [Test]
-        public void TestTxAllModes()
-        {
-            TestTxAllModes(false);
-
-            TestTxAllModes(true);
-
-            Console.WriteLine("Done");
-        }
-
-        protected void TestTxAllModes(bool withTimeout)
-        {
-            if (!TxEnabled())
-                return;
-
-            var cache = Cache();
-
-            int cntr = 0;
-
-            foreach (TransactionConcurrency concurrency in Enum.GetValues(typeof(TransactionConcurrency))) {
-                foreach (TransactionIsolation isolation in Enum.GetValues(typeof(TransactionIsolation))) {
-                    Console.WriteLine("Test tx [concurrency=" + concurrency + ", isolation=" + isolation + "]");
-
-                    ITransaction tx = Transactions.Tx;
-
-                    Assert.IsNull(tx);
-
-                    tx = withTimeout 
-                        ? Transactions.TxStart(concurrency, isolation, TimeSpan.FromMilliseconds(1100), 10)
-                        : Transactions.TxStart(concurrency, isolation);
-
-                    Assert.AreEqual(concurrency, tx.Concurrency);
-                    Assert.AreEqual(isolation, tx.Isolation);
-
-                    if (withTimeout)
-                        Assert.AreEqual(1100, tx.Timeout.TotalMilliseconds);
-
-                    try {
-                        cache.Put(1, cntr);
-
-                        tx.Commit();
-                    }
-                    finally {
-                        tx.Dispose();
-                    }
-
-                    tx = Transactions.Tx;
-
-                    Assert.IsNull(tx);
-
-                    Assert.AreEqual(cntr, cache.Get(1));
-
-                    cntr++;
-                }
-            }
-        }
-
-        [Test]
-        public void TestTxAttributes()
-        {
-            if (!TxEnabled())
-                return;
-
-            ITransaction tx = Transactions.TxStart(TransactionConcurrency.Optimistic,
-                TransactionIsolation.RepeatableRead, TimeSpan.FromMilliseconds(2500), 100);
-
-            Assert.IsFalse(tx.IsRollbackOnly);
-            Assert.AreEqual(TransactionConcurrency.Optimistic, tx.Concurrency);
-            Assert.AreEqual(TransactionIsolation.RepeatableRead, tx.Isolation);
-            Assert.AreEqual(2500, tx.Timeout.TotalMilliseconds);
-            Assert.AreEqual(TransactionState.Active, tx.State);
-            Assert.IsTrue(tx.StartTime.Ticks > 0);
-            Assert.AreEqual(tx.NodeId, GetIgnite(0).GetCluster().GetLocalNode().Id);
-
-            DateTime startTime1 = tx.StartTime;
-
-            tx.Commit();
-
-            Assert.IsFalse(tx.IsRollbackOnly);
-            Assert.AreEqual(TransactionState.Committed, tx.State);
-            Assert.AreEqual(TransactionConcurrency.Optimistic, tx.Concurrency);
-            Assert.AreEqual(TransactionIsolation.RepeatableRead, tx.Isolation);
-            Assert.AreEqual(2500, tx.Timeout.TotalMilliseconds);
-            Assert.AreEqual(startTime1, tx.StartTime);
-
-            Thread.Sleep(100);
-
-            tx = Transactions.TxStart(TransactionConcurrency.Pessimistic, TransactionIsolation.ReadCommitted,
-                TimeSpan.FromMilliseconds(3500), 200);
-
-            Assert.IsFalse(tx.IsRollbackOnly);
-            Assert.AreEqual(TransactionConcurrency.Pessimistic, tx.Concurrency);
-            Assert.AreEqual(TransactionIsolation.ReadCommitted, tx.Isolation);
-            Assert.AreEqual(3500, tx.Timeout.TotalMilliseconds);
-            Assert.AreEqual(TransactionState.Active, tx.State);
-            Assert.IsTrue(tx.StartTime.Ticks > 0);
-            Assert.IsTrue(tx.StartTime > startTime1);
-
-            DateTime startTime2 = tx.StartTime;
-
-            tx.Rollback();
-
-            Assert.AreEqual(TransactionState.RolledBack, tx.State);
-            Assert.AreEqual(TransactionConcurrency.Pessimistic, tx.Concurrency);
-            Assert.AreEqual(TransactionIsolation.ReadCommitted, tx.Isolation);
-            Assert.AreEqual(3500, tx.Timeout.TotalMilliseconds);
-            Assert.AreEqual(startTime2, tx.StartTime);
-
-            Thread.Sleep(100);
-
-            tx = Transactions.TxStart(TransactionConcurrency.Optimistic, TransactionIsolation.RepeatableRead,
-                TimeSpan.FromMilliseconds(2500), 100);
-
-            Assert.IsFalse(tx.IsRollbackOnly);
-            Assert.AreEqual(TransactionConcurrency.Optimistic, tx.Concurrency);
-            Assert.AreEqual(TransactionIsolation.RepeatableRead, tx.Isolation);
-            Assert.AreEqual(2500, tx.Timeout.TotalMilliseconds);
-            Assert.AreEqual(TransactionState.Active, tx.State);
-            Assert.IsTrue(tx.StartTime > startTime2);
-
-            DateTime startTime3 = tx.StartTime;
-
-            tx.Commit();
-
-            Assert.IsFalse(tx.IsRollbackOnly);
-            Assert.AreEqual(TransactionState.Committed, tx.State);
-            Assert.AreEqual(TransactionConcurrency.Optimistic, tx.Concurrency);
-            Assert.AreEqual(TransactionIsolation.RepeatableRead, tx.Isolation);
-            Assert.AreEqual(2500, tx.Timeout.TotalMilliseconds);
-            Assert.AreEqual(startTime3, tx.StartTime);
-        }
-
-        [Test]
-        public void TestTxRollbackOnly()
-        {
-            if (!TxEnabled())
-                return;
-
-            var cache = Cache();
-
-            cache.Put(1, 1);
-
-            cache.Put(2, 2);
-
-            ITransaction tx = Transactions.TxStart();
-
-            cache.Put(1, 10);
-
-            cache.Put(2, 20);
-
-            Assert.IsFalse(tx.IsRollbackOnly);
-
-            tx.SetRollbackonly();
-
-            Assert.IsTrue(tx.IsRollbackOnly);
-
-            Assert.AreEqual(TransactionState.MarkedRollback, tx.State);
-
-            try
-            {
-                tx.Commit();
-
-                Assert.Fail("Commit must fail.");
-            }
-            catch (IgniteException e)
-            {
-                Console.WriteLine("Expected exception: " + e);
-            }
-
-            tx.Dispose();
-
-            Assert.AreEqual(TransactionState.RolledBack, tx.State);
-
-            Assert.IsTrue(tx.IsRollbackOnly);
-
-            Assert.AreEqual(1, cache.Get(1));
-
-            Assert.AreEqual(2, cache.Get(2));
-
-            tx = Transactions.Tx;
-
-            Assert.IsNull(tx);
-        }
-
-        [Test]
-        public void TestTxMetrics()
-        {
-            if (!TxEnabled())
-                return;
-
-            var cache = Cache();
-            
-            var startTime = DateTime.UtcNow.AddSeconds(-1);
-
-            Transactions.ResetMetrics();
-
-            var metrics = Transactions.GetMetrics();
-            
-            Assert.AreEqual(0, metrics.TxCommits);
-            Assert.AreEqual(0, metrics.TxRollbacks);
-
-            using (Transactions.TxStart())
-            {
-                cache.Put(1, 1);
-            }
-            
-            using (var tx = Transactions.TxStart())
-            {
-                cache.Put(1, 1);
-                tx.Commit();
-            }
-
-            metrics = Transactions.GetMetrics();
-
-            Assert.AreEqual(1, metrics.TxCommits);
-            Assert.AreEqual(1, metrics.TxRollbacks);
-
-            Assert.LessOrEqual(startTime, metrics.CommitTime);
-            Assert.LessOrEqual(startTime, metrics.RollbackTime);
-
-            Assert.GreaterOrEqual(DateTime.UtcNow, metrics.CommitTime);
-            Assert.GreaterOrEqual(DateTime.UtcNow, metrics.RollbackTime);
-        }
-
-        [Test]
-        public void TestTxStateAndExceptions()
-        {
-            if (!TxEnabled())
-                return;
-
-            var tx = Transactions.TxStart();
-
-            Assert.AreEqual(TransactionState.Active, tx.State);
-            Assert.AreEqual(Thread.CurrentThread.ManagedThreadId, tx.ThreadId);
-
-            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);
-
-            try
-            {
-                tx.Commit();
-                Assert.Fail();
-            }
-            catch (InvalidOperationException)
-            {
-                // Expected
-            }
-
-            tx = Transactions.TxStart();
-
-            Assert.AreEqual(TransactionState.Active, tx.State);
-
-            tx.CommitAsync().Wait();
-
-            Assert.AreEqual(TransactionState.Committed, tx.State);
-
-            var task = tx.RollbackAsync();  // Illegal, but should not fail here; will fail in task
-
-            try
-            {
-                task.Wait();
-                Assert.Fail();
-            }
-            catch (AggregateException)
-            {
-                // 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>
         [Test]
@@ -3404,16 +2789,6 @@ namespace Apache.Ignite.Core.Tests.Cache
             return false;
         }
 
-        protected virtual bool TxEnabled()
-        {
-            return true;
-        }
-
-        protected bool LockingEnabled()
-        {
-            return TxEnabled();
-        }
-
         protected virtual bool LocalCache()
         {
             return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/864af7eb/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTransactionalTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTransactionalTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTransactionalTest.cs
new file mode 100644
index 0000000..e836ba2
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTransactionalTest.cs
@@ -0,0 +1,556 @@
+/*
+ * 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;
+    using System.Linq;
+    using System.Threading;
+    using System.Threading.Tasks;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Transactions;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Transactional cache tests.
+    /// </summary>
+    public abstract class CacheAbstractTransactionalTest : CacheAbstractTest
+    {
+        /// <summary>
+        /// Simple cache lock test (while <see cref="TestLock"/> is ignored).
+        /// </summary>
+        [Test]
+        public void TestLockSimple()
+        {
+            var cache = Cache();
+
+            const int key = 7;
+
+            Action<ICacheLock> checkLock = lck =>
+            {
+                using (lck)
+                {
+                    Assert.Throws<InvalidOperationException>(lck.Exit); // can't exit if not entered
+
+                    lck.Enter();
+
+                    Assert.IsTrue(cache.IsLocalLocked(key, true));
+                    Assert.IsTrue(cache.IsLocalLocked(key, false));
+
+                    lck.Exit();
+
+                    Assert.IsFalse(cache.IsLocalLocked(key, true));
+                    Assert.IsFalse(cache.IsLocalLocked(key, false));
+
+                    Assert.IsTrue(lck.TryEnter());
+
+                    Assert.IsTrue(cache.IsLocalLocked(key, true));
+                    Assert.IsTrue(cache.IsLocalLocked(key, false));
+
+                    lck.Exit();
+                }
+
+                Assert.Throws<ObjectDisposedException>(lck.Enter); // Can't enter disposed lock
+            };
+
+            checkLock(cache.Lock(key));
+            checkLock(cache.LockAll(new[] { key, 1, 2, 3 }));
+        }
+
+        /// <summary>
+        /// Tests cache locks.
+        /// </summary>
+        [Test]
+        [Ignore("IGNITE-835")]
+        public void TestLock()
+        {
+            var cache = Cache();
+
+            const int key = 7;
+
+            // Lock
+            CheckLock(cache, key, () => cache.Lock(key));
+
+            // LockAll
+            CheckLock(cache, key, () => cache.LockAll(new[] { key, 2, 3, 4, 5 }));
+        }
+
+        /// <summary>
+        /// Internal lock test routine.
+        /// </summary>
+        /// <param name="cache">Cache.</param>
+        /// <param name="key">Key.</param>
+        /// <param name="getLock">Function to get the lock.</param>
+        private static void CheckLock(ICache<int, int> cache, int key, Func<ICacheLock> getLock)
+        {
+            var sharedLock = getLock();
+
+            using (sharedLock)
+            {
+                Assert.Throws<InvalidOperationException>(() => sharedLock.Exit());  // can't exit if not entered
+
+                sharedLock.Enter();
+
+                try
+                {
+                    Assert.IsTrue(cache.IsLocalLocked(key, true));
+                    Assert.IsTrue(cache.IsLocalLocked(key, false));
+
+                    EnsureCannotLock(getLock, sharedLock);
+
+                    sharedLock.Enter();
+
+                    try
+                    {
+                        Assert.IsTrue(cache.IsLocalLocked(key, true));
+                        Assert.IsTrue(cache.IsLocalLocked(key, false));
+
+                        EnsureCannotLock(getLock, sharedLock);
+                    }
+                    finally
+                    {
+                        sharedLock.Exit();
+                    }
+
+                    Assert.IsTrue(cache.IsLocalLocked(key, true));
+                    Assert.IsTrue(cache.IsLocalLocked(key, false));
+
+                    EnsureCannotLock(getLock, sharedLock);
+
+                    Assert.Throws<SynchronizationLockException>(() => sharedLock.Dispose()); // can't dispose while locked
+                }
+                finally
+                {
+                    sharedLock.Exit();
+                }
+
+                Assert.IsFalse(cache.IsLocalLocked(key, true));
+                Assert.IsFalse(cache.IsLocalLocked(key, false));
+
+                var innerTask = new Task(() =>
+                {
+                    Assert.IsTrue(sharedLock.TryEnter());
+                    sharedLock.Exit();
+
+                    using (var otherLock = getLock())
+                    {
+                        Assert.IsTrue(otherLock.TryEnter());
+                        otherLock.Exit();
+                    }
+                });
+
+                innerTask.Start();
+                innerTask.Wait();
+            }
+
+            Assert.IsFalse(cache.IsLocalLocked(key, true));
+            Assert.IsFalse(cache.IsLocalLocked(key, false));
+
+            var outerTask = new Task(() =>
+            {
+                using (var otherLock = getLock())
+                {
+                    Assert.IsTrue(otherLock.TryEnter());
+                    otherLock.Exit();
+                }
+            });
+
+            outerTask.Start();
+            outerTask.Wait();
+
+            Assert.Throws<ObjectDisposedException>(() => sharedLock.Enter());  // Can't enter disposed lock
+        }
+
+        /// <summary>
+        /// Ensure that lock cannot be obtained by other threads.
+        /// </summary>
+        /// <param name="getLock">Get lock function.</param>
+        /// <param name="sharedLock">Shared lock.</param>
+        private static void EnsureCannotLock(Func<ICacheLock> getLock, ICacheLock sharedLock)
+        {
+            var task = new Task(() =>
+            {
+                Assert.IsFalse(sharedLock.TryEnter());
+                Assert.IsFalse(sharedLock.TryEnter(TimeSpan.FromMilliseconds(100)));
+
+                using (var otherLock = getLock())
+                {
+                    Assert.IsFalse(otherLock.TryEnter());
+                    Assert.IsFalse(otherLock.TryEnter(TimeSpan.FromMilliseconds(100)));
+                }
+            });
+
+            task.Start();
+            task.Wait();
+        }
+
+        /// <summary>
+        /// Tests that commit applies cache changes.
+        /// </summary>
+        [Test]
+        public void TestTxCommit([Values(true, false)] bool async)
+        {
+            var cache = Cache();
+
+            Assert.IsNull(Transactions.Tx);
+
+            using (var tx = Transactions.TxStart())
+            {
+                cache.Put(1, 1);
+                cache.Put(2, 2);
+
+                if (async)
+                {
+                    var task = tx.CommitAsync();
+
+                    task.Wait();
+
+                    Assert.IsTrue(task.IsCompleted);
+                }
+                else
+                    tx.Commit();
+            }
+
+            Assert.AreEqual(1, cache.Get(1));
+            Assert.AreEqual(2, cache.Get(2));
+
+            Assert.IsNull(Transactions.Tx);
+        }
+
+        /// <summary>
+        /// Tests that rollback reverts cache changes.
+        /// </summary>
+        [Test]
+        public void TestTxRollback()
+        {
+            var cache = Cache();
+
+            cache.Put(1, 1);
+            cache.Put(2, 2);
+
+            Assert.IsNull(Transactions.Tx);
+
+            using (var tx = Transactions.TxStart())
+            {
+                cache.Put(1, 10);
+                cache.Put(2, 20);
+
+                tx.Rollback();
+            }
+
+            Assert.AreEqual(1, cache.Get(1));
+            Assert.AreEqual(2, cache.Get(2));
+
+            Assert.IsNull(Transactions.Tx);
+        }
+
+        /// <summary>
+        /// Tests that Dispose without Commit reverts changes.
+        /// </summary>
+        [Test]
+        public void TestTxClose()
+        {
+            var cache = Cache();
+
+            cache.Put(1, 1);
+            cache.Put(2, 2);
+
+            Assert.IsNull(Transactions.Tx);
+
+            using (Transactions.TxStart())
+            {
+                cache.Put(1, 10);
+                cache.Put(2, 20);
+            }
+
+            Assert.AreEqual(1, cache.Get(1));
+            Assert.AreEqual(2, cache.Get(2));
+
+            Assert.IsNull(Transactions.Tx);
+        }
+
+        /// <summary>
+        /// Tests all concurrency and isolation modes with and without timeout.
+        /// </summary>
+        [Test]
+        public void TestTxAllModes([Values(true, false)] bool withTimeout)
+        {
+            var cache = Cache();
+
+            int cntr = 0;
+
+            foreach (TransactionConcurrency concurrency in Enum.GetValues(typeof(TransactionConcurrency)))
+            {
+                foreach (TransactionIsolation isolation in Enum.GetValues(typeof(TransactionIsolation)))
+                {
+                    Console.WriteLine("Test tx [concurrency=" + concurrency + ", isolation=" + isolation + "]");
+
+                    Assert.IsNull(Transactions.Tx);
+
+                    using (var tx = withTimeout
+                        ? Transactions.TxStart(concurrency, isolation, TimeSpan.FromMilliseconds(1100), 10)
+                        : Transactions.TxStart(concurrency, isolation))
+                    {
+
+                        Assert.AreEqual(concurrency, tx.Concurrency);
+                        Assert.AreEqual(isolation, tx.Isolation);
+
+                        if (withTimeout)
+                            Assert.AreEqual(1100, tx.Timeout.TotalMilliseconds);
+
+                        cache.Put(1, cntr);
+
+                        tx.Commit();
+                    }
+
+                    Assert.IsNull(Transactions.Tx);
+
+                    Assert.AreEqual(cntr, cache.Get(1));
+
+                    cntr++;
+                }
+            }
+        }
+
+        /// <summary>
+        /// Tests that transaction properties are applied and propagated properly.
+        /// </summary>
+        [Test]
+        public void TestTxAttributes()
+        {
+            ITransaction tx = Transactions.TxStart(TransactionConcurrency.Optimistic,
+                TransactionIsolation.RepeatableRead, TimeSpan.FromMilliseconds(2500), 100);
+
+            Assert.IsFalse(tx.IsRollbackOnly);
+            Assert.AreEqual(TransactionConcurrency.Optimistic, tx.Concurrency);
+            Assert.AreEqual(TransactionIsolation.RepeatableRead, tx.Isolation);
+            Assert.AreEqual(2500, tx.Timeout.TotalMilliseconds);
+            Assert.AreEqual(TransactionState.Active, tx.State);
+            Assert.IsTrue(tx.StartTime.Ticks > 0);
+            Assert.AreEqual(tx.NodeId, GetIgnite(0).GetCluster().GetLocalNode().Id);
+
+            DateTime startTime1 = tx.StartTime;
+
+            tx.Commit();
+
+            Assert.IsFalse(tx.IsRollbackOnly);
+            Assert.AreEqual(TransactionState.Committed, tx.State);
+            Assert.AreEqual(TransactionConcurrency.Optimistic, tx.Concurrency);
+            Assert.AreEqual(TransactionIsolation.RepeatableRead, tx.Isolation);
+            Assert.AreEqual(2500, tx.Timeout.TotalMilliseconds);
+            Assert.AreEqual(startTime1, tx.StartTime);
+
+            Thread.Sleep(100);
+
+            tx = Transactions.TxStart(TransactionConcurrency.Pessimistic, TransactionIsolation.ReadCommitted,
+                TimeSpan.FromMilliseconds(3500), 200);
+
+            Assert.IsFalse(tx.IsRollbackOnly);
+            Assert.AreEqual(TransactionConcurrency.Pessimistic, tx.Concurrency);
+            Assert.AreEqual(TransactionIsolation.ReadCommitted, tx.Isolation);
+            Assert.AreEqual(3500, tx.Timeout.TotalMilliseconds);
+            Assert.AreEqual(TransactionState.Active, tx.State);
+            Assert.IsTrue(tx.StartTime.Ticks > 0);
+            Assert.IsTrue(tx.StartTime > startTime1);
+
+            DateTime startTime2 = tx.StartTime;
+
+            tx.Rollback();
+
+            Assert.AreEqual(TransactionState.RolledBack, tx.State);
+            Assert.AreEqual(TransactionConcurrency.Pessimistic, tx.Concurrency);
+            Assert.AreEqual(TransactionIsolation.ReadCommitted, tx.Isolation);
+            Assert.AreEqual(3500, tx.Timeout.TotalMilliseconds);
+            Assert.AreEqual(startTime2, tx.StartTime);
+
+            Thread.Sleep(100);
+
+            tx = Transactions.TxStart(TransactionConcurrency.Optimistic, TransactionIsolation.RepeatableRead,
+                TimeSpan.FromMilliseconds(2500), 100);
+
+            Assert.IsFalse(tx.IsRollbackOnly);
+            Assert.AreEqual(TransactionConcurrency.Optimistic, tx.Concurrency);
+            Assert.AreEqual(TransactionIsolation.RepeatableRead, tx.Isolation);
+            Assert.AreEqual(2500, tx.Timeout.TotalMilliseconds);
+            Assert.AreEqual(TransactionState.Active, tx.State);
+            Assert.IsTrue(tx.StartTime > startTime2);
+
+            DateTime startTime3 = tx.StartTime;
+
+            tx.Commit();
+
+            Assert.IsFalse(tx.IsRollbackOnly);
+            Assert.AreEqual(TransactionState.Committed, tx.State);
+            Assert.AreEqual(TransactionConcurrency.Optimistic, tx.Concurrency);
+            Assert.AreEqual(TransactionIsolation.RepeatableRead, tx.Isolation);
+            Assert.AreEqual(2500, tx.Timeout.TotalMilliseconds);
+            Assert.AreEqual(startTime3, tx.StartTime);
+        }
+
+        /// <summary>
+        /// Tests <see cref="ITransaction.IsRollbackOnly"/> flag.
+        /// </summary>
+        [Test]
+        public void TestTxRollbackOnly()
+        {
+            var cache = Cache();
+
+            cache.Put(1, 1);
+            cache.Put(2, 2);
+
+            var tx = Transactions.TxStart();
+
+            cache.Put(1, 10);
+            cache.Put(2, 20);
+
+            Assert.IsFalse(tx.IsRollbackOnly);
+
+            tx.SetRollbackonly();
+
+            Assert.IsTrue(tx.IsRollbackOnly);
+
+            Assert.AreEqual(TransactionState.MarkedRollback, tx.State);
+
+            var ex = Assert.Throws<TransactionRollbackException>(() => tx.Commit());
+            Assert.IsTrue(ex.Message.StartsWith("Invalid transaction state for prepare [state=MARKED_ROLLBACK"));
+
+            tx.Dispose();
+
+            Assert.AreEqual(TransactionState.RolledBack, tx.State);
+
+            Assert.IsTrue(tx.IsRollbackOnly);
+
+            Assert.AreEqual(1, cache.Get(1));
+            Assert.AreEqual(2, cache.Get(2));
+
+            Assert.IsNull(Transactions.Tx);
+        }
+
+        /// <summary>
+        /// Tests transaction metrics.
+        /// </summary>
+        [Test]
+        public void TestTxMetrics()
+        {
+            var cache = Cache();
+
+            var startTime = DateTime.UtcNow.AddSeconds(-1);
+
+            Transactions.ResetMetrics();
+
+            var metrics = Transactions.GetMetrics();
+
+            Assert.AreEqual(0, metrics.TxCommits);
+            Assert.AreEqual(0, metrics.TxRollbacks);
+
+            using (Transactions.TxStart())
+            {
+                cache.Put(1, 1);
+            }
+
+            using (var tx = Transactions.TxStart())
+            {
+                cache.Put(1, 1);
+                tx.Commit();
+            }
+
+            metrics = Transactions.GetMetrics();
+
+            Assert.AreEqual(1, metrics.TxCommits);
+            Assert.AreEqual(1, metrics.TxRollbacks);
+
+            Assert.LessOrEqual(startTime, metrics.CommitTime);
+            Assert.LessOrEqual(startTime, metrics.RollbackTime);
+
+            Assert.GreaterOrEqual(DateTime.UtcNow, metrics.CommitTime);
+            Assert.GreaterOrEqual(DateTime.UtcNow, metrics.RollbackTime);
+        }
+
+        /// <summary>
+        /// Tests transaction state transitions.
+        /// </summary>
+        [Test]
+        public void TestTxStateAndExceptions()
+        {
+            var tx = Transactions.TxStart();
+
+            Assert.AreEqual(TransactionState.Active, tx.State);
+            Assert.AreEqual(Thread.CurrentThread.ManagedThreadId, tx.ThreadId);
+
+            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);
+
+            Assert.Throws<InvalidOperationException>(() => tx.Commit());
+
+            tx = Transactions.TxStart();
+
+            Assert.AreEqual(TransactionState.Active, tx.State);
+
+            tx.CommitAsync().Wait();
+
+            Assert.AreEqual(TransactionState.Committed, tx.State);
+
+            var task = tx.RollbackAsync();  // Illegal, but should not fail here; will fail in task
+
+            Assert.Throws<AggregateException>(() => task.Wait());
+        }
+
+        /// <summary>
+        /// Tests the transaction deadlock detection.
+        /// </summary>
+        [Test]
+        public void TestTxDeadlockDetection()
+        {
+            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);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/864af7eb/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheLocalAtomicTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheLocalAtomicTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheLocalAtomicTest.cs
index b60c254..2c8c070 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheLocalAtomicTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheLocalAtomicTest.cs
@@ -39,11 +39,6 @@ namespace Apache.Ignite.Core.Tests.Cache
             return false;
         }
 
-        protected override bool TxEnabled()
-        {
-            return false;
-        }
-
         protected override bool LocalCache()
         {
             return true;

http://git-wip-us.apache.org/repos/asf/ignite/blob/864af7eb/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheLocalTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheLocalTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheLocalTest.cs
index 02cb987..000218b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheLocalTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheLocalTest.cs
@@ -17,7 +17,7 @@
 
 namespace Apache.Ignite.Core.Tests.Cache
 {
-    public class CacheLocalTest : CacheAbstractTest
+    public class CacheLocalTest : CacheAbstractTransactionalTest
     {
         protected override int CachePartitions()
         {
@@ -39,10 +39,6 @@ namespace Apache.Ignite.Core.Tests.Cache
             return false;
         }
 
-        protected override bool TxEnabled()
-        {
-            return true;
-        }
         protected override bool LocalCache()
         {
             return true;

http://git-wip-us.apache.org/repos/asf/ignite/blob/864af7eb/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedAtomicNearEnabledTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedAtomicNearEnabledTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedAtomicNearEnabledTest.cs
index 4f6e7a0..77cc77f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedAtomicNearEnabledTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedAtomicNearEnabledTest.cs
@@ -37,11 +37,6 @@ namespace Apache.Ignite.Core.Tests.Cache
             return true;
         }
 
-        protected override bool TxEnabled()
-        {
-            return false;
-        }
-
         protected override int Backups()
         {
             return 1;

http://git-wip-us.apache.org/repos/asf/ignite/blob/864af7eb/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedAtomicTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedAtomicTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedAtomicTest.cs
index ab59c64..e6f851e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedAtomicTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedAtomicTest.cs
@@ -37,11 +37,6 @@ namespace Apache.Ignite.Core.Tests.Cache
             return false;
         }
 
-        protected override bool TxEnabled()
-        {
-            return false;
-        }
-
         protected override int Backups()
         {
             return 1;

http://git-wip-us.apache.org/repos/asf/ignite/blob/864af7eb/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedNearEnabledTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedNearEnabledTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedNearEnabledTest.cs
index 830698b..81d2e5a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedNearEnabledTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedNearEnabledTest.cs
@@ -20,7 +20,7 @@ namespace Apache.Ignite.Core.Tests.Cache
     using NUnit.Framework;
 
     [Category(TestUtils.CategoryIntensive)]
-    public class CachePartitionedNearEnabledTest : CacheAbstractTest
+    public class CachePartitionedNearEnabledTest : CacheAbstractTransactionalTest
     {
         protected override int GridCount()
         {
@@ -37,11 +37,6 @@ namespace Apache.Ignite.Core.Tests.Cache
             return true;
         }
 
-        protected override bool TxEnabled()
-        {
-            return true;
-        }
-
         protected override int Backups()
         {
             return 1;

http://git-wip-us.apache.org/repos/asf/ignite/blob/864af7eb/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedTest.cs
index 02d3208..68546b9 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CachePartitionedTest.cs
@@ -20,7 +20,7 @@ namespace Apache.Ignite.Core.Tests.Cache
     using NUnit.Framework;
 
     [Category(TestUtils.CategoryIntensive)]
-    public class CachePartitionedTest : CacheAbstractTest
+    public class CachePartitionedTest : CacheAbstractTransactionalTest
     {
         protected override int GridCount()
         {
@@ -37,11 +37,6 @@ namespace Apache.Ignite.Core.Tests.Cache
             return false;
         }
 
-        protected override bool TxEnabled()
-        {
-            return true;
-        }
-
         protected override int Backups()
         {
             return 1;

http://git-wip-us.apache.org/repos/asf/ignite/blob/864af7eb/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheReplicatedAtomicTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheReplicatedAtomicTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheReplicatedAtomicTest.cs
index db6f5a5..bd31c77 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheReplicatedAtomicTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheReplicatedAtomicTest.cs
@@ -42,11 +42,6 @@ namespace Apache.Ignite.Core.Tests.Cache
             return false;
         }
 
-        protected override bool TxEnabled()
-        {
-            return false;
-        }
-
         protected override int Backups()
         {
             return GridCount() - 1;

http://git-wip-us.apache.org/repos/asf/ignite/blob/864af7eb/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheReplicatedTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheReplicatedTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheReplicatedTest.cs
index 7c70222..2d1c257 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheReplicatedTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheReplicatedTest.cs
@@ -20,7 +20,7 @@ namespace Apache.Ignite.Core.Tests.Cache
     using NUnit.Framework;
 
     [Category(TestUtils.CategoryIntensive)]
-    public class CacheReplicatedTest : CacheAbstractTest
+    public class CacheReplicatedTest : CacheAbstractTransactionalTest
     {
         protected override int CachePartitions()
         {
@@ -42,11 +42,6 @@ namespace Apache.Ignite.Core.Tests.Cache
             return false;
         }
 
-        protected override bool TxEnabled()
-        {
-            return true;
-        }
-
         protected override int Backups()
         {
             return GridCount() - 1;


[47/50] [abbrv] ignite git commit: IGNITE-4307 .NET: Fix AtomicConfiguration in AtomicSequenceExample

Posted by yz...@apache.org.
IGNITE-4307 .NET: Fix AtomicConfiguration in AtomicSequenceExample


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 06ef846067411350081c4112ef9e9466deb6086c
Parents: 8dd0322
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Wed Dec 28 11:29:30 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Wed Dec 28 11:29:30 2016 +0300

----------------------------------------------------------------------
 .../examples/Apache.Ignite.Examples/App.config  |  2 ++
 .../DataStructures/AtomicSequenceExample.cs     | 26 +++-----------------
 2 files changed, 6 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/06ef8460/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 8b16df3..13f0d86 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/App.config
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/App.config
@@ -40,6 +40,8 @@
                 <string>Apache.Ignite.ExamplesDll.Compute.AverageSalaryJob</string>
             </types>
         </binaryConfiguration>
+
+        <atomicConfiguration atomicSequenceReserveSize="10" />
         
         <discoverySpi type="TcpDiscoverySpi">
             <ipFinder type="TcpDiscoveryMulticastIpFinder">

http://git-wip-us.apache.org/repos/asf/ignite/blob/06ef8460/modules/platforms/dotnet/examples/Apache.Ignite.Examples/DataStructures/AtomicSequenceExample.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/DataStructures/AtomicSequenceExample.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/DataStructures/AtomicSequenceExample.cs
index f08c998..89e8ed3 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/DataStructures/AtomicSequenceExample.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/DataStructures/AtomicSequenceExample.cs
@@ -21,9 +21,6 @@ namespace Apache.Ignite.Examples.DataStructures
     using System.Threading;
     using Apache.Ignite.Core;
     using Apache.Ignite.Core.DataStructures;
-    using Apache.Ignite.Core.DataStructures.Configuration;
-    using Apache.Ignite.Core.Discovery.Tcp;
-    using Apache.Ignite.Core.Discovery.Tcp.Multicast;
     using Apache.Ignite.ExamplesDll.DataStructures;
 
     /// <summary>
@@ -44,26 +41,11 @@ namespace Apache.Ignite.Examples.DataStructures
         [STAThread]
         public static void Main()
         {
-            var atomicCfg = new AtomicConfiguration
-            {
-                // Each node reserves 10 numbers to itself, so that 10 increments can be done locally, 
-                // without communicating to other nodes. After that, another 10 elements are reserved.
-                AtomicSequenceReserveSize = 10
-            };
-
-            var cfg = new IgniteConfiguration
-            {
-                DiscoverySpi = new TcpDiscoverySpi
-                {
-                    IpFinder = new TcpDiscoveryMulticastIpFinder
-                    {
-                        Endpoints = new[] { "127.0.0.1:47500" }
-                    }
-                },
-                AtomicConfiguration = atomicCfg
-            };
+            // See app.config: <atomicConfiguration atomicSequenceReserveSize="10" />
+            // Each node reserves 10 numbers to itself, so that 10 increments can be done locally, 
+            // without communicating to other nodes. After that, another 10 elements are reserved.
 
-            using (var ignite = Ignition.Start(cfg))
+            using (var ignite = Ignition.StartFromApplicationConfiguration())
             {
                 Console.WriteLine();
                 Console.WriteLine(">>> Atomic sequence example started.");


[06/50] [abbrv] ignite git commit: Fixed 'singleRmv' flag for cache.remove(k, v) operation in tx cache.

Posted by yz...@apache.org.
Fixed 'singleRmv' flag for cache.remove(k, v) operation in tx cache.

(cherry picked from commit 126ab60)


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 800579c9ca193f7660e35f241c255904c573eef1
Parents: e82eefe
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 20 15:15:04 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 20 15:18:28 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/cache/GridCacheAdapter.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/800579c9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index f235f6a..3b4557d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -2948,6 +2948,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /**
      * @param key Key.
+     * @param filter Filter.
      * @return {@code True} if entry was removed.
      * @throws IgniteCheckedException If failed.
      */
@@ -2959,7 +2960,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                     Collections.singletonList(key),
                     /*retval*/false,
                     filter,
-                    /*singleRmv*/true).get().success();
+                    /*singleRmv*/filter == null).get().success();
             }
 
             @Override public String toString() {


[26/50] [abbrv] ignite git commit: Web console beta-7.

Posted by yz...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/sql/sql.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/sql/sql.controller.js b/modules/web-console/frontend/app/modules/sql/sql.controller.js
index 4e6e372..0d0b171 100644
--- a/modules/web-console/frontend/app/modules/sql/sql.controller.js
+++ b/modules/web-console/frontend/app/modules/sql/sql.controller.js
@@ -50,6 +50,9 @@ class Paragraph {
         const self = this;
 
         self.id = 'paragraph-' + paragraphId++;
+        self.qryType = paragraph.qryType || 'query';
+        self.maxPages = 0;
+        self.filter = '';
 
         _.assign(this, paragraph);
 
@@ -77,27 +80,28 @@ class Paragraph {
             enableColumnMenus: false,
             flatEntityAccess: true,
             fastWatch: true,
+            categories: [],
             rebuildColumns() {
                 if (_.isNil(this.api))
                     return;
 
-                this.categories = [];
+                this.categories.length = 0;
+
                 this.columnDefs = _.reduce(self.meta, (cols, col, idx) => {
-                    if (self.columnFilter(col)) {
-                        cols.push({
-                            displayName: col.fieldName,
-                            headerTooltip: _fullColName(col),
-                            field: idx.toString(),
-                            minWidth: 50,
-                            cellClass: 'cell-left'
-                        });
+                    cols.push({
+                        displayName: col.fieldName,
+                        headerTooltip: _fullColName(col),
+                        field: idx.toString(),
+                        minWidth: 50,
+                        cellClass: 'cell-left',
+                        visible: self.columnFilter(col)
+                    });
 
-                        this.categories.push({
-                            name: col.fieldName,
-                            visible: true,
-                            selectable: true
-                        });
-                    }
+                    this.categories.push({
+                        name: col.fieldName,
+                        visible: self.columnFilter(col),
+                        selectable: true
+                    });
 
                     return cols;
                 }, []);
@@ -182,8 +186,8 @@ class Paragraph {
 }
 
 // Controller for SQL notebook screen.
-export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', '$animate', '$location', '$anchorScroll', '$state', '$filter', '$modal', '$popover', 'IgniteLoading', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteAgentMonitor', 'IgniteChartColors', 'IgniteNotebook', 'IgniteScanFilterInput', 'IgniteNodes', 'uiGridExporterConstants', 'IgniteVersion',
-    function($root, $scope, $http, $q, $timeout, $interval, $animate, $location, $anchorScroll, $state, $filter, $modal, $popover, Loading, LegacyUtils, Messages, Confirm, agentMonitor, IgniteChartColors, Notebook, ScanFilterInput, Nodes, uiGridExporterConstants, Version) {
+export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', '$animate', '$location', '$anchorScroll', '$state', '$filter', '$modal', '$popover', 'IgniteLoading', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteAgentMonitor', 'IgniteChartColors', 'IgniteNotebook', 'IgniteNodes', 'uiGridExporterConstants', 'IgniteVersion',
+    function($root, $scope, $http, $q, $timeout, $interval, $animate, $location, $anchorScroll, $state, $filter, $modal, $popover, Loading, LegacyUtils, Messages, Confirm, agentMonitor, IgniteChartColors, Notebook, Nodes, uiGridExporterConstants, Version) {
         let stopTopology = null;
 
         const _tryStopRefresh = function(paragraph) {
@@ -206,6 +210,15 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
         $scope.caches = [];
 
         $scope.pageSizes = [50, 100, 200, 400, 800, 1000];
+        $scope.maxPages = [
+            {label: 'Unlimited', value: 0},
+            {label: '1', value: 1},
+            {label: '5', value: 5},
+            {label: '10', value: 10},
+            {label: '20', value: 20},
+            {label: '50', value: 50},
+            {label: '100', value: 100}
+        ];
 
         $scope.timeLineSpans = ['1', '5', '10', '15', '30'];
 
@@ -213,7 +226,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
 
         $scope.modes = LegacyUtils.mkOptions(['PARTITIONED', 'REPLICATED', 'LOCAL']);
 
-        $scope.loadingText = $root.IgniteDemoMode ? 'Demo grid is starting. Please wait...' : 'Loading notebook screen...';
+        $scope.loadingText = $root.IgniteDemoMode ? 'Demo grid is starting. Please wait...' : 'Loading query notebook screen...';
 
         $scope.timeUnit = [
             {value: 1000, label: 'seconds', short: 's'},
@@ -768,11 +781,10 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
 
             if (idx >= 0) {
                 if (!_.includes($scope.notebook.expandedParagraphs, idx))
-                    $scope.notebook.expandedParagraphs.push(idx);
+                    $scope.notebook.expandedParagraphs = $scope.notebook.expandedParagraphs.concat([idx]);
 
-                setTimeout(function() {
-                    $scope.notebook.paragraphs[idx].ace.focus();
-                });
+                if ($scope.notebook.paragraphs[idx].ace)
+                    setTimeout(() => $scope.notebook.paragraphs[idx].ace.focus());
             }
 
             $location.hash(id);
@@ -816,7 +828,8 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                             let item = _.find(cachesAcc, {name: cache.name});
 
                             if (_.isNil(item)) {
-                                cache.label = maskCacheName(cache.name);
+                                cache.label = maskCacheName(cache.name, true);
+                                cache.value = cache.name;
 
                                 cache.nodes = [];
 
@@ -839,7 +852,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                         return;
 
                     // Reset to first cache in case of stopped selected.
-                    const cacheNames = _.map($scope.caches, (cache) => cache.name);
+                    const cacheNames = _.map($scope.caches, (cache) => cache.value);
 
                     _.forEach($scope.notebook.paragraphs, (paragraph) => {
                         if (!_.includes(cacheNames, paragraph.cacheName))
@@ -885,7 +898,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                     (paragraph) => new Paragraph($animate, $timeout, paragraph));
 
                 if (_.isEmpty($scope.notebook.paragraphs))
-                    $scope.addParagraph();
+                    $scope.addQuery();
                 else
                     $scope.rebuildScrollParagraphs();
             })
@@ -936,32 +949,37 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                 paragraph.edit = false;
         };
 
-        $scope.addParagraph = function() {
+        $scope.addParagraph = (paragraph, sz) => {
+            if ($scope.caches && $scope.caches.length > 0)
+                paragraph.cacheName = _.head($scope.caches).value;
+
+            $scope.notebook.paragraphs.push(paragraph);
+
+            $scope.notebook.expandedParagraphs.push(sz);
+
+            $scope.rebuildScrollParagraphs();
+
+            $location.hash(paragraph.id);
+        };
+
+        $scope.addQuery = function() {
             const sz = $scope.notebook.paragraphs.length;
 
             const paragraph = new Paragraph($animate, $timeout, {
                 name: 'Query' + (sz === 0 ? '' : sz),
                 query: '',
-                pageSize: $scope.pageSizes[0],
+                pageSize: $scope.pageSizes[1],
                 timeLineSpan: $scope.timeLineSpans[0],
                 result: 'none',
                 rate: {
                     value: 1,
                     unit: 60000,
                     installed: false
-                }
+                },
+                qryType: 'query'
             });
 
-            if ($scope.caches && $scope.caches.length > 0)
-                paragraph.cacheName = $scope.caches[0].name;
-
-            $scope.notebook.paragraphs.push(paragraph);
-
-            $scope.notebook.expandedParagraphs.push(sz);
-
-            $scope.rebuildScrollParagraphs();
-
-            $location.hash(paragraph.id);
+            $scope.addParagraph(paragraph, sz);
 
             $timeout(() => {
                 $anchorScroll();
@@ -970,6 +988,26 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
             });
         };
 
+        $scope.addScan = function() {
+            const sz = $scope.notebook.paragraphs.length;
+
+            const paragraph = new Paragraph($animate, $timeout, {
+                name: 'Scan' + (sz === 0 ? '' : sz),
+                query: '',
+                pageSize: $scope.pageSizes[1],
+                timeLineSpan: $scope.timeLineSpans[0],
+                result: 'none',
+                rate: {
+                    value: 1,
+                    unit: 60000,
+                    installed: false
+                },
+                qryType: 'scan'
+            });
+
+            $scope.addParagraph(paragraph, sz);
+        };
+
         function _saveChartSettings(paragraph) {
             if (!_.isEmpty(paragraph.charts)) {
                 const chart = paragraph.charts[0].api.getScope().chart;
@@ -1010,7 +1048,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
         };
 
         $scope.removeParagraph = function(paragraph) {
-            Confirm.confirm('Are you sure you want to remove: "' + paragraph.name + '"?')
+            Confirm.confirm('Are you sure you want to remove query: "' + paragraph.name + '"?')
                 .then(function() {
                     $scope.stopRefresh(paragraph);
 
@@ -1315,8 +1353,8 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
             return false;
         };
 
-        $scope.execute = (paragraph, nonCollocatedJoins = false) => {
-            const local = !!paragraph.localQry;
+        $scope.execute = (paragraph, local = false) => {
+            const nonCollocatedJoins = !!paragraph.nonCollocatedJoins;
 
             $scope.actionAvailable(paragraph, true) && _chooseNode(paragraph.cacheName, local)
                 .then((nid) => {
@@ -1330,16 +1368,16 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                     return _closeOldQuery(paragraph)
                         .then(() => {
                             const args = paragraph.queryArgs = {
+                                type: 'QUERY',
                                 cacheName: paragraph.cacheName,
-                                pageSize: paragraph.pageSize,
                                 query: paragraph.query,
-                                firstPageOnly: paragraph.firstPageOnly,
+                                pageSize: paragraph.pageSize,
+                                maxPages: paragraph.maxPages,
                                 nonCollocatedJoins,
-                                type: 'QUERY',
                                 localNid: local ? nid : null
                             };
 
-                            const qry = args.firstPageOnly ? addLimit(args.query, args.pageSize) : paragraph.query;
+                            const qry = args.maxPages ? addLimit(args.query, args.pageSize * args.maxPages) : paragraph.query;
 
                             return agentMonitor.query(nid, args.cacheName, qry, nonCollocatedJoins, local, args.pageSize);
                         })
@@ -1386,10 +1424,10 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                 .then(() => _chooseNode(paragraph.cacheName, false))
                 .then((nid) => {
                     const args = paragraph.queryArgs = {
+                        type: 'EXPLAIN',
                         cacheName: paragraph.cacheName,
-                        pageSize: paragraph.pageSize,
                         query: 'EXPLAIN ' + paragraph.query,
-                        type: 'EXPLAIN'
+                        pageSize: paragraph.pageSize
                     };
 
                     return agentMonitor.query(nid, args.cacheName, args.query, false, false, args.pageSize);
@@ -1403,8 +1441,10 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                 .then(() => paragraph.ace.focus());
         };
 
-        $scope.scan = (paragraph, query = null) => {
-            const local = !!paragraph.localQry;
+        $scope.scan = (paragraph, local = false) => {
+            const {filter, caseSensitive} = paragraph;
+            const prefix = caseSensitive ? SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE : SCAN_CACHE_WITH_FILTER;
+            const query = `${prefix}${filter}`;
 
             $scope.actionAvailable(paragraph, false) && _chooseNode(paragraph.cacheName, local)
                 .then((nid) => {
@@ -1418,45 +1458,22 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                     _closeOldQuery(paragraph)
                         .then(() => {
                             const args = paragraph.queryArgs = {
+                                type: 'SCAN',
                                 cacheName: paragraph.cacheName,
-                                pageSize: paragraph.pageSize,
-                                firstPageOnly: paragraph.firstPageOnly,
                                 query,
-                                type: 'SCAN',
+                                filter,
+                                pageSize: paragraph.pageSize,
                                 localNid: local ? nid : null
                             };
 
                             return agentMonitor.query(nid, args.cacheName, query, false, local, args.pageSize);
                         })
-                        .then((res) => {
-                            if (paragraph.firstPageOnly) {
-                                res.hasMore = false;
-
-                                _processQueryResult(paragraph, true, res);
-
-                                _closeOldQuery(paragraph);
-                            }
-                            else
-                                _processQueryResult(paragraph, true, res);
-                        })
+                        .then((res) => _processQueryResult(paragraph, true, res))
                         .catch((err) => {
                             paragraph.errMsg = err.message;
 
                             _showLoading(paragraph, false);
-                        })
-                        .then(() => paragraph.ace.focus());
-                });
-        };
-
-        $scope.scanWithFilter = (paragraph) => {
-            if (!$scope.actionAvailable(paragraph, false))
-                return;
-
-            ScanFilterInput.open()
-                .then(({filter, caseSensitive}) => {
-                    const prefix = caseSensitive ? SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE : SCAN_CACHE_WITH_FILTER;
-
-                    $scope.scan(paragraph, `${prefix}${filter}`);
+                        });
                 });
         };
 
@@ -1511,25 +1528,23 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
 
                     _showLoading(paragraph, false);
                 })
-                .then(() => paragraph.ace.focus());
+                .then(() => paragraph.ace && paragraph.ace.focus());
         };
 
-        const _export = (fileName, columnFilter, meta, rows) => {
+        const _export = (fileName, columnDefs, meta, rows) => {
             let csvContent = '';
 
             const cols = [];
             const excludedCols = [];
 
-            if (meta) {
-                _.forEach(meta, (col, idx) => {
-                    if (columnFilter(col))
-                        cols.push(_fullColName(col));
-                    else
-                        excludedCols.push(idx);
-                });
+            _.forEach(meta, (col, idx) => {
+                if (columnDefs[idx].visible)
+                    cols.push(_fullColName(col));
+                else
+                    excludedCols.push(idx);
+            });
 
-                csvContent += cols.join(';') + '\n';
-            }
+            csvContent += cols.join(';') + '\n';
 
             _.forEach(rows, (row) => {
                 cols.length = 0;
@@ -1543,8 +1558,8 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                     });
                 }
                 else {
-                    _.forEach(meta, (col) => {
-                        if (columnFilter(col)) {
+                    _.forEach(columnDefs, (col) => {
+                        if (col.visible) {
                             const elem = row[col.fieldName];
 
                             cols.push(_.isUndefined(elem) ? '' : JSON.stringify(elem));
@@ -1559,7 +1574,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
         };
 
         $scope.exportCsv = function(paragraph) {
-            _export(paragraph.name + '.csv', paragraph.columnFilter, paragraph.meta, paragraph.rows);
+            _export(paragraph.name + '.csv', paragraph.gridOptions.columnDefs, paragraph.meta, paragraph.rows);
 
             // paragraph.gridOptions.api.exporter.csvExport(uiGridExporterConstants.ALL, uiGridExporterConstants.VISIBLE);
         };
@@ -1573,17 +1588,17 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
 
             return Promise.resolve(args.localNid || _chooseNode(args.cacheName, false))
                 .then((nid) => agentMonitor.queryGetAll(nid, args.cacheName, args.query, !!args.nonCollocatedJoins, !!args.localNid))
-                .then((res) => _export(paragraph.name + '-all.csv', paragraph.columnFilter, res.columns, res.rows))
+                .then((res) => _export(paragraph.name + '-all.csv', paragraph.gridOptions.columnDefs, res.columns, res.rows))
                 .catch(Messages.showError)
-                .then(() => paragraph.ace.focus());
+                .then(() => paragraph.ace && paragraph.ace.focus());
         };
 
         // $scope.exportPdfAll = function(paragraph) {
         //    $http.post('/api/v1/agent/query/getAll', {query: paragraph.query, cacheName: paragraph.cacheName})
-        //        .success(function(item) {
-        //            _export(paragraph.name + '-all.csv', item.meta, item.rows);
+        //    .then(({data}) {
+        //        _export(paragraph.name + '-all.csv', data.meta, data.rows);
         //    })
-        //    .error(Messages.showError);
+        //    .catch(Messages.showError);
         // };
 
         $scope.rateAsString = function(paragraph) {
@@ -1652,9 +1667,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
         $scope.dblclickMetadata = function(paragraph, node) {
             paragraph.ace.insert(node.name);
 
-            setTimeout(function() {
-                paragraph.ace.focus();
-            }, 1);
+            setTimeout(() => paragraph.ace.focus(), 1);
         };
 
         $scope.importMetadata = function() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/sql/sql.module.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/sql/sql.module.js b/modules/web-console/frontend/app/modules/sql/sql.module.js
index d615d28..a1ffde9 100644
--- a/modules/web-console/frontend/app/modules/sql/sql.module.js
+++ b/modules/web-console/frontend/app/modules/sql/sql.module.js
@@ -19,7 +19,6 @@ import angular from 'angular';
 
 import NotebookData from './Notebook.data';
 import Notebook from './Notebook.service';
-import ScanFilterInput from './scan-filter-input.service';
 import notebook from './notebook.controller';
 import sql from './sql.controller';
 
@@ -55,6 +54,5 @@ angular.module('ignite-console.sql', [
     )
     .service('IgniteNotebookData', NotebookData)
     .service('IgniteNotebook', Notebook)
-    .service('IgniteScanFilterInput', ScanFilterInput)
     .controller('notebookController', notebook)
     .controller('sqlController', sql);

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration.state.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration.state.js b/modules/web-console/frontend/app/modules/states/configuration.state.js
index 888c804..61dca13 100644
--- a/modules/web-console/frontend/app/modules/states/configuration.state.js
+++ b/modules/web-console/frontend/app/modules/states/configuration.state.js
@@ -24,12 +24,14 @@ import previewPanel from './configuration/preview-panel.directive.js';
 import ConfigurationSummaryCtrl from './configuration/summary/summary.controller';
 import ConfigurationResource from './configuration/Configuration.resource';
 import summaryTabs from './configuration/summary/summary-tabs.directive';
+import IgniteSummaryZipper from './configuration/summary/summary-zipper.service';
 
 angular.module('ignite-console.states.configuration', ['ui.router'])
     .directive(...previewPanel)
     // Summary screen
     .directive(...summaryTabs)
     // Services.
+    .service('IgniteSummaryZipper', IgniteSummaryZipper)
     .service('IgniteConfigurationResource', ConfigurationResource)
     // Configure state provider.
     .config(['$stateProvider', 'AclRouteProvider', ($stateProvider, AclRoute) => {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade
index b34aba0..bcac5ad 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade
@@ -54,6 +54,6 @@ include /app/helpers/jade/mixins.jade
                         -var required = nodeFilterKind + ' === "Custom"'
 
                         +java-class('Class name:', customNodeFilter + '.className', '"customNodeFilter"',
-                            'true', required, 'Class name of custom node filter implementation')
+                            'true', required, 'Class name of custom node filter implementation', required)
             .col-sm-6
                 +preview-xml-java(model, 'cacheNodeFilter', 'igfss')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade
index 5062ce1..cfbaf12 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade
@@ -53,6 +53,9 @@ include /app/helpers/jade/mixins.jade
                     +number('Long query timeout:', model + '.longQueryWarningTimeout', '"longQueryWarningTimeout"', 'true', '3000', '0',
                         'Timeout in milliseconds after which long query warning will be printed')
                 .settings-row
+                    +number('History size:', model + '.queryDetailMetricsSize', '"queryDetailMetricsSize"', 'true', '0', '0',
+                        'Size of queries detail metrics that will be stored in memory for monitoring purposes')
+                .settings-row
                     -var form = 'querySqlFunctionClasses';
                     -var sqlFunctionClasses = model + '.sqlFunctionClasses';
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade
index 1cf80b8..ea350f2 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade
@@ -102,9 +102,9 @@ mixin hibernateField(name, model, items, valid, save, newItem)
                                         'Parallel load cache minimum threshold.<br/>\
                                         If <b>0</b> then load sequentially.')
                                 .details-row
-                                    +java-class('Hasher', pojoStoreFactory + '.hasher', '"pojoHasher"', 'true', 'false', 'Hash calculator')
+                                    +java-class('Hasher', pojoStoreFactory + '.hasher', '"pojoHasher"', 'true', 'false', 'Hash calculator', required)
                                 .details-row
-                                    +java-class('Transformer', pojoStoreFactory + '.transformer', '"pojoTransformer"', 'true', 'false', 'Types transformer')
+                                    +java-class('Transformer', pojoStoreFactory + '.transformer', '"pojoTransformer"', 'true', 'false', 'Types transformer', required)
                                 .details-row
                                     +checkbox('Escape table and filed names', pojoStoreFactory + '.sqlEscapeAll', '"sqlEscapeAll"',
                                         'If enabled than all schema, table and field names will be escaped with double quotes (for example: "tableName"."fieldName").<br/>\

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade
index 5cc996d..259909e 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade
@@ -19,6 +19,7 @@ include /app/helpers/jade/mixins.jade
 -var form = 'checkpoint'
 -var model = 'backupItem.checkpointSpi'
 -var CustomCheckpoint = 'model.kind === "Custom"'
+-var CacheCheckpoint = 'model.kind === "Cache"'
 
 .panel.panel-default(ng-form=form novalidate)
     .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")')
@@ -44,7 +45,7 @@ include /app/helpers/jade/mixins.jade
                         .group-content(ng-show='#{model} && #{model}.length > 0' ng-repeat='model in #{model} track by $index')
                             hr(ng-if='$index != 0')
                             .settings-row
-                                +dropdown-required('Checkpoint SPI:', 'model.kind', '"checkpointKind" + $index', 'true', 'true', 'Choose checkpoint configuration variant', '[\
+                                +dropdown-required-autofocus('Checkpoint SPI:', 'model.kind', '"checkpointKind" + $index', 'true', 'true', 'Choose checkpoint configuration variant', '[\
                                         {value: "FS", label: "File System"},\
                                         {value: "Cache", label: "Cache"},\
                                         {value: "S3", label: "Amazon S3"},\
@@ -64,13 +65,13 @@ include /app/helpers/jade/mixins.jade
                             div(ng-show='model.kind === "FS"')
                                 include ./checkpoint/fs.jade
 
-                            div(ng-show='model.kind === "Cache"')
+                            div(ng-show=CacheCheckpoint)
                                 .settings-row
-                                    +dropdown-required-empty('Cache:', 'model.Cache.cache', '"checkpointCacheCache"+ $index', 'true', 'true',
+                                    +dropdown-required-empty('Cache:', 'model.Cache.cache', '"checkpointCacheCache"+ $index', 'true', CacheCheckpoint,
                                         'Choose cache', 'No caches configured for current cluster', 'clusterCaches', 'Cache to use for storing checkpoints')
                                 .settings-row
                                     +java-class('Listener:', 'model.Cache.checkpointListener', '"checkpointCacheListener" + $index', 'true', 'false',
-                                        'Checkpoint listener implementation class name')
+                                        'Checkpoint listener implementation class name', CacheCheckpoint)
 
                             div(ng-show='model.kind === "S3"')
                                 include ./checkpoint/s3.jade
@@ -80,6 +81,6 @@ include /app/helpers/jade/mixins.jade
 
                             .settings-row(ng-show=CustomCheckpoint)
                                 +java-class('Class name:', 'model.Custom.className', '"checkpointCustomClassName" + $index', 'true', CustomCheckpoint,
-                                'Custom CheckpointSpi implementation class')
+                                'Custom CheckpointSpi implementation class', CustomCheckpoint)
             .col-sm-6
                 +preview-xml-java('backupItem', 'clusterCheckpoint', 'caches')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade
index efb6ad0..6ec4535 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade
@@ -36,13 +36,13 @@ include /app/helpers/jade/mixins.jade
             -var valid = form + '[' + name + '].$valid'
             -var save = dirPaths + '[$index] = ' + model
 
-            div(ng-repeat='model in #{dirPaths} track by $index' ng-init='obj = {}')
+            div(ng-repeat='item in #{dirPaths} track by $index' ng-init='obj = {}')
                 label.col-xs-12.col-sm-12.col-md-12
                     .indexField
                         | {{ $index+1 }})
-                    +table-remove-conditional-button(dirPaths, 'true', 'Remove path')
+                    +table-remove-conditional-button(dirPaths, 'true', 'Remove path', 'item')
                     span(ng-hide='field.edit')
-                        a.labelFormField(ng-click='(field.edit = true) && (#{model} = model)') {{ model }}
+                        a.labelFormField(ng-click='(field.edit = true) && (#{model} = item)') {{ item }}
                     span(ng-if='field.edit')
                         +table-text-field(name, model, dirPaths, valid, save, 'Input directory path', false)
                             +table-save-button(valid, save, false)
@@ -63,4 +63,4 @@ include /app/helpers/jade/mixins.jade
 
 .settings-row
     +java-class('Listener:', 'model.FS.checkpointListener', '"checkpointFsListener" + $index', 'true', 'false',
-        'Checkpoint listener implementation class name')
+        'Checkpoint listener implementation class name', 'model.kind === "FS"')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade
index 874799c..5a13337 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade
@@ -16,15 +16,17 @@
 
 include /app/helpers/jade/mixins.jade
 
+-var jdbcCheckpoint = 'model.kind === "JDBC"'
+
 .settings-row
-    +text('Data source bean name:', 'model.JDBC.dataSourceBean', '"checkpointJdbcDataSourceBean" + $index', 'model.kind === "JDBC"', 'Input bean name',
+    +text('Data source bean name:', 'model.JDBC.dataSourceBean', '"checkpointJdbcDataSourceBean" + $index', jdbcCheckpoint, 'Input bean name',
     'Name of the data source bean in Spring context')
 .settings-row
-    +dialect('Dialect:', 'model.JDBC.dialect', '"checkpointJdbcDialect" + $index', 'model.kind === "JDBC"',
+    +dialect('Dialect:', 'model.JDBC.dialect', '"checkpointJdbcDialect" + $index', jdbcCheckpoint,
     'Dialect of SQL implemented by a particular RDBMS:', 'Generic JDBC dialect', 'Choose JDBC dialect')
 .settings-row
     +java-class('Listener:', 'model.JDBC.checkpointListener', '"checkpointJdbcListener" + $index', 'true', 'false',
-        'Checkpoint listener implementation class name')
+        'Checkpoint listener implementation class name', jdbcCheckpoint)
 +showHideLink('jdbcExpanded', 'settings')
     .details-row
         +text('User:', 'model.JDBC.user', '"checkpointJdbcUser" + $index', 'false', 'Input user name', 'Checkpoint jdbc user name')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade
index da28da7..6531897 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade
@@ -18,16 +18,17 @@ include /app/helpers/jade/mixins.jade
 
 -var credentialsModel = 'model.S3.awsCredentials'
 -var clientCfgModel = 'model.S3.clientConfiguration'
--var checkpointS3Path = 'model.S3.awsCredentials.kind === "Properties"'
--var checkpointS3Custom = 'model.S3.awsCredentials.kind === "Custom"'
+-var checkpointS3 = 'model.kind === "S3"'
+-var checkpointS3Path = checkpointS3 + ' && model.S3.awsCredentials.kind === "Properties"'
+-var checkpointS3Custom = checkpointS3 + ' && model.S3.awsCredentials.kind === "Custom"'
 
 -var clientRetryModel = clientCfgModel + '.retryPolicy'
--var checkpointS3DefaultMaxRetry = clientRetryModel + '.kind === "DefaultMaxRetries"'
--var checkpointS3DynamoDbMaxRetry = clientRetryModel + '.kind === "DynamoDBMaxRetries"'
--var checkpointS3CustomRetry = clientRetryModel + '.kind === "Custom"'
+-var checkpointS3DefaultMaxRetry = checkpointS3 + ' && ' + clientRetryModel + '.kind === "DefaultMaxRetries"'
+-var checkpointS3DynamoDbMaxRetry = checkpointS3 + ' && ' + clientRetryModel + '.kind === "DynamoDBMaxRetries"'
+-var checkpointS3CustomRetry = checkpointS3 + ' && ' + clientRetryModel + '.kind === "Custom"'
 
 .settings-row
-    +dropdown-required('AWS credentials:', 'model.S3.awsCredentials.kind', '"checkpointS3AwsCredentials"', 'true', 'model.kind === "S3"', 'Custom', '[\
+    +dropdown-required('AWS credentials:', 'model.S3.awsCredentials.kind', '"checkpointS3AwsCredentials"', 'true', checkpointS3, 'Custom', '[\
         {value: "Basic", label: "Basic"},\
         {value: "Properties", label: "Properties"},\
         {value: "Anonymous", label: "Anonymous"},\
@@ -51,12 +52,12 @@ include /app/helpers/jade/mixins.jade
 .panel-details(ng-show=checkpointS3Custom)
     .details-row
         +java-class('Class name:', credentialsModel + '.Custom.className', '"checkpointS3CustomClassName" + $index', 'true', checkpointS3Custom,
-        'Custom AWS credentials provider implementation class')
+        'Custom AWS credentials provider implementation class', checkpointS3Custom)
 .settings-row
     +text('Bucket name suffix:', 'model.S3.bucketNameSuffix', '"checkpointS3BucketNameSuffix"', 'false', 'default-bucket', 'Bucket name suffix')
 .settings-row
     +java-class('Listener:', 'model.S3.checkpointListener', '"checkpointS3Listener" + $index', 'true', 'false',
-        'Checkpoint listener implementation class name')
+        'Checkpoint listener implementation class name', checkpointS3)
 +showHideLink('s3Expanded', 'client configuration')
     .details-row
         +dropdown('Protocol:', clientCfgModel + '.protocol', '"checkpointS3Protocol"', 'true', 'HTTPS', '[\
@@ -121,10 +122,10 @@ include /app/helpers/jade/mixins.jade
     .panel-details(ng-show=checkpointS3CustomRetry)
         .details-row
             +java-class('Retry condition:', clientRetryModel + '.Custom.retryCondition', '"checkpointS3CustomRetryPolicy" + $index', 'true', checkpointS3CustomRetry,
-            'Retry condition on whether a specific request and exception should be retried')
+            'Retry condition on whether a specific request and exception should be retried', checkpointS3CustomRetry)
         .details-row
             +java-class('Backoff strategy:', clientRetryModel + '.Custom.backoffStrategy', '"checkpointS3CustomBackoffStrategy" + $index', 'true', checkpointS3CustomRetry,
-            'Back-off strategy for controlling how long the next retry should wait')
+            'Back-off strategy for controlling how long the next retry should wait', checkpointS3CustomRetry)
         .details-row
             +number-required('Maximum retry attempts:', clientRetryModel + '.Custom.maxErrorRetry', '"checkpointS3CustomMaxErrorRetry"', 'true', checkpointS3CustomRetry, '-1', '1',
             'Maximum number of retry attempts for failed requests')
@@ -159,13 +160,13 @@ include /app/helpers/jade/mixins.jade
         'Maximum amount of time that an idle connection may sit in the connection pool and still be eligible for reuse')
     .details-row
         +java-class('DNS resolver:', clientCfgModel + '.dnsResolver', '"checkpointS3DnsResolver" + $index', 'true', 'false',
-        'DNS Resolver that should be used to for resolving AWS IP addresses')
+        'DNS Resolver that should be used to for resolving AWS IP addresses', checkpointS3)
     .details-row
         +number('Response metadata cache size:', clientCfgModel + '.responseMetadataCacheSize', '"checkpointS3ResponseMetadataCacheSize"', 'true', '50', '0',
         'Response metadata cache size')
     .details-row
         +java-class('SecureRandom class name:', clientCfgModel + '.secureRandom', '"checkpointS3SecureRandom" + $index', 'true', 'false',
-        'SecureRandom to be used by the SDK class name')
+        'SecureRandom to be used by the SDK class name', checkpointS3)
     .details-row
         +checkbox('Use reaper', clientCfgModel + '.useReaper', '"checkpointS3UseReaper"', 'Checks if the IdleConnectionReaper is to be started')
     .details-row

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade
index 31a6be7..8e77ac4 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade
@@ -21,4 +21,4 @@ include /app/helpers/jade/mixins.jade
 
 div
     .details-row
-        +java-class('Class:', model + '.class', '"collisionCustom"', 'true', required, 'CollisionSpi implementation class')
+        +java-class('Class:', model + '.class', '"collisionCustom"', 'true', required, 'CollisionSpi implementation class', required)

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade
index d4e537a..dbe0478 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade
@@ -37,7 +37,7 @@ div
             'Node should attempt to steal jobs from other nodes')
     .details-row
         +java-class('External listener:', model + '.externalCollisionListener', '"jsExternalCollisionListener"', 'true', 'false',
-            'Listener to be set for notification of external collision events')
+            'Listener to be set for notification of external collision events', 'backupItem.collision.kind === "JobStealing"')
     .details-row
         +ignite-form-group
             ignite-form-field-label

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade
index 4cfd9f5..aa99b49 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade
@@ -18,8 +18,14 @@ include /app/helpers/jade/mixins.jade
 
 -var form = 'deployment'
 -var model = 'backupItem'
+-var modelDeployment = 'backupItem.deploymentSpi'
 -var exclude = model + '.peerClassLoadingLocalClassPathExclude'
 -var enabled = 'backupItem.peerClassLoadingEnabled'
+-var uriListModel = modelDeployment + '.URI.uriList'
+-var scannerModel = modelDeployment + '.URI.scanners'
+-var uriDeployment = modelDeployment + '.kind === "URI"'
+-var localDeployment = modelDeployment + '.kind === "Local"'
+-var customDeployment = modelDeployment + '.kind === "Custom"'
 
 .panel.panel-default(ng-form=form novalidate)
     .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")')
@@ -57,7 +63,7 @@ include /app/helpers/jade/mixins.jade
                 .settings-row
                     +number('Pool size:', model + '.peerClassLoadingThreadPoolSize', '"peerClassLoadingThreadPoolSize"', enabled, '2', '1', 'Thread pool size to use for peer class loading')
                 .settings-row
-                    +ignite-form-group(ng-model=exclude ng-form=form)
+                    +ignite-form-group
                         -var uniqueTip = 'Such package already exists'
 
                         ignite-form-field-label
@@ -81,7 +87,7 @@ include /app/helpers/jade/mixins.jade
                                             | {{ $index+1 }})
                                         +table-remove-button(exclude, 'Remove package name')
                                         span(ng-hide='field.edit')
-                                            a.labelFormField(ng-click='#{enabled} && (field.edit = true) && (#{model} = model)') {{ model }}
+                                            a.labelFormField(ng-click='(field.edit = true) && (#{model} = model)') {{ model }}
                                         span(ng-if='field.edit')
                                             +table-java-package-field(name, model, exclude, valid, save, false)
                                                 +table-save-button(valid, save, false)
@@ -107,8 +113,125 @@ include /app/helpers/jade/mixins.jade
                                         +table-save-button(valid, save, true)
                                         +unique-feedback(name, uniqueTip)
 
-
                         .group-content-empty(ng-if='!(#{exclude}.length) && !group.add.length')
                             | Not defined
+                .settings-row
+                    +dropdown('Deployment variant:', modelDeployment + '.kind', '"deploymentKind"', 'true', 'Default',
+                        '[\
+                            {value: "URI", label: "URI"},\
+                            {value: "Local", label: "Local"}, \
+                            {value: "Custom", label: "Custom"},\
+                            {value: undefined, label: "Default"}\
+                        ]',
+                        'Grid deployment SPI is in charge of deploying tasks and classes from different sources:\
+                        <ul>\
+                            <li>URI - Deploy tasks from different sources like file system folders, email and HTTP</li>\
+                            <li>Local - Only within VM deployment on local node</li>\
+                            <li>Custom - Custom implementation of DeploymentSpi</li>\
+                            <li>Default - Default configuration of LocalDeploymentSpi will be used</li>\
+                        </ul>')
+                .panel-details(ng-show=uriDeployment)
+                    .details-row
+                        +ignite-form-group()
+                            -var uniqueTip = 'Such URI already configured'
+
+                            ignite-form-field-label
+                                | URI list
+                            ignite-form-group-tooltip
+                                | List of URI which point to GAR file and which should be scanned by SPI for the new tasks
+                            ignite-form-group-add(ng-click='(group.add = [{}])')
+                                | Add URI.
+
+                            .group-content(ng-if=uriListModel + '.length')
+                                -var model = 'obj.model';
+                                -var name = '"edit" + $index'
+                                -var valid = form + '[' + name + '].$valid'
+                                -var save = uriListModel + '[$index] = ' + model
+
+                                div(ng-repeat='model in #{uriListModel} track by $index' ng-init='obj = {}')
+                                    label.col-xs-12.col-sm-12.col-md-12
+                                        .indexField
+                                            | {{ $index+1 }})
+                                        +table-remove-button(uriListModel, 'Remove URI')
+                                        span(ng-hide='field.edit')
+                                            a.labelFormField(ng-click='(field.edit = true) && (#{model} = model)') {{ model }}
+                                        span(ng-if='field.edit')
+                                            +table-url-field(name, model, uriListModel, valid, save, false)
+                                                +table-save-button(valid, save, false)
+                                                +unique-feedback(name, uniqueTip)
+
+                            .group-content(ng-repeat='field in group.add')
+                                -var model = 'new';
+                                -var name = '"new"'
+                                -var valid = form + '[' + name + '].$valid'
+                                -var save = uriListModel + '.push(' + model + ')'
+
+                                div(type='internal' name='URI')
+                                    label.col-xs-12.col-sm-12.col-md-12
+                                        +table-url-field(name, model, uriListModel, valid, save, true)
+                                            +table-save-button(valid, save, true)
+                                            +unique-feedback(name, uniqueTip)
+
+                            .group-content-empty(ng-if='!(#{uriListModel}.length) && !group.add.length')
+                                | Not defined
+                    .details-row
+                        +text('Temporary directory path:', modelDeployment + '.URI.temporaryDirectoryPath', '"DeploymentURITemporaryDirectoryPath"', 'false', 'Temporary directory path',
+                        'Absolute path to temporary directory which will be used by deployment SPI to keep all deployed classes in')
+                    .details-row
+                        +ignite-form-group()
+                            -var uniqueTip = 'Such scanner already configured'
+
+                            ignite-form-field-label
+                                | Scanner list
+                            ignite-form-group-tooltip
+                                | List of URI deployment scanners
+                            ignite-form-group-add(ng-click='(group.add = [{}])')
+                                | Add scanner
+
+                            .group-content(ng-if=scannerModel + '.length')
+                                -var model = 'obj.model';
+                                -var name = '"edit" + $index'
+                                -var valid = form + '[' + name + '].$valid'
+                                -var save = scannerModel + '[$index] = ' + model
+
+                                div(ng-repeat='model in #{scannerModel} track by $index' ng-init='obj = {}')
+                                    label.col-xs-12.col-sm-12.col-md-12
+                                        .indexField
+                                            | {{ $index+1 }})
+                                        +table-remove-button(scannerModel, 'Remove scanner')
+                                        span(ng-hide='field.edit')
+                                            a.labelFormField(ng-click='(field.edit = true) && (#{model} = model)') {{ model }}
+                                        span(ng-if='field.edit')
+                                            +table-java-class-field('Scanner:', name, model, scannerModel, valid, save, false)
+                                                +table-save-button(valid, save, false)
+                                                +unique-feedback(name, uniqueTip)
+
+                            .group-content(ng-repeat='field in group.add')
+                                -var model = 'new';
+                                -var name = '"new"'
+                                -var valid = form + '[' + name + '].$valid'
+                                -var save = scannerModel + '.push(' + model + ')'
+
+                                div(type='internal' name='Scanner')
+                                    label.col-xs-12.col-sm-12.col-md-12
+                                        // (lbl, name, model, items, valid, save, newItem)
+                                        +table-java-class-field('Scanner:', name, model, scannerModel, valid, save, true)
+                                            +table-save-button(valid, save, true)
+                                            +unique-feedback(name, uniqueTip)
+
+                            .group-content-empty(ng-if='!(#{scannerModel}.length) && !group.add.length')
+                                | Not defined
+                    .details-row
+                        +java-class('Listener:', modelDeployment + '.URI.listener', '"DeploymentURIListener"', 'true', 'false', 'Deployment event listener', uriDeployment)
+                    .details-row
+                        +checkbox('Check MD5', modelDeployment + '.URI.checkMd5', '"DeploymentURICheckMd5"', 'Exclude files with same md5s from deployment')
+                    .details-row
+                        +checkbox('Encode URI', modelDeployment + '.URI.encodeUri', '"DeploymentURIEncodeUri"', 'URI must be encoded before usage')
+                .panel-details(ng-show=localDeployment)
+                    .details-row
+                        +java-class('Listener:', modelDeployment + '.Local.listener', '"DeploymentLocalListener"', 'true', 'false', 'Deployment event listener', localDeployment)
+                .panel-details(ng-show=customDeployment)
+                    .details-row
+                        +java-class('Class:', modelDeployment + '.Custom.className', '"DeploymentCustom"', 'true', customDeployment, 'DeploymentSpi implementation class', customDeployment)
             .col-sm-6
                 +preview-xml-java(model, 'clusterDeployment')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade
index 3f2d6cb..643ea97 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade
@@ -59,10 +59,10 @@ include /app/helpers/jade/mixins.jade
                     .settings-row
                         +java-class('Filter:', modelEventStorage + '.Memory.filter', '"EventStorageFilter"', 'true', 'false',
                         'Filter for events to be recorded<br/>\
-                        Should be implementation of o.a.i.lang.IgnitePredicate&lt;o.a.i.events.Event&gt;')
+                        Should be implementation of o.a.i.lang.IgnitePredicate&lt;o.a.i.events.Event&gt;', eventStorageMemory)
 
                 .settings-row(ng-show=eventStorageCustom)
-                    +java-class('Class:', modelEventStorage + '.Custom.className', '"EventStorageCustom"', 'true', eventStorageCustom, 'Event storage implementation class name')
+                    +java-class('Class:', modelEventStorage + '.Custom.className', '"EventStorageCustom"', 'true', eventStorageCustom, 'Event storage implementation class name', eventStorageCustom)
 
             .col-sm-6
                 +preview-xml-java(model, 'clusterEvents')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade
index aaed8e9..1665659 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade
@@ -45,7 +45,7 @@ include /app/helpers/jade/mixins.jade
                         .group-content(ng-show='#{failoverSpi} && #{failoverSpi}.length > 0' ng-repeat='model in #{failoverSpi} track by $index')
                             hr(ng-if='$index != 0')
                             .settings-row
-                                +dropdown-required('Failover SPI:', 'model.kind', '"failoverKind" + $index', 'true', 'true', 'Choose Failover SPI', '[\
+                                +dropdown-required-autofocus('Failover SPI:', 'model.kind', '"failoverKind" + $index', 'true', 'true', 'Choose Failover SPI', '[\
                                         {value: "JobStealing", label: "Job stealing"},\
                                         {value: "Never", label: "Never"},\
                                         {value: "Always", label: "Always"},\
@@ -68,6 +68,6 @@ include /app/helpers/jade/mixins.jade
                                     'Maximum number of attempts to execute a failed job on another node')
                             .settings-row(ng-show=failoverCustom)
                                 +java-class('SPI implementation', 'model.Custom.class', '"failoverSpiClass" + $index', 'true', failoverCustom,
-                                    'Custom FailoverSpi implementation class name.')
+                                    'Custom FailoverSpi implementation class name.', failoverCustom)
             .col-sm-6
                 +preview-xml-java(model, 'clusterFailover')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade
index 2e567ed..48b1776 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade
@@ -27,7 +27,7 @@ div
         +java-class('Curator:', model + '.curator', '"curator"', 'true', 'false',
             'The Curator framework in use<br/>\
             By default generates curator of org.apache.curator. framework.imps.CuratorFrameworkImpl\
-            class with configured connect string, retry policy, and default session and connection timeouts')
+            class with configured connect string, retry policy, and default session and connection timeouts', required)
     .details-row
         +text('Connect string:', model + '.zkConnectionString', '"' + discoveryKind + 'ConnectionString"', required, 'host:port[chroot][,host:port[chroot]]',
             'When "IGNITE_ZK_CONNECTION_STRING" system property is not configured this property will be used')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade
index 5a03de8..5db89f5 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade
@@ -21,4 +21,4 @@ include /app/helpers/jade/mixins.jade
 -var required = 'backupItem.discovery.kind === "ZooKeeper" && backupItem.discovery.ZooKeeper.retryPolicy.kind === "Custom"'
 
 .details-row
-    +java-class('Class name:', retry + '.className', '"customClassName"', 'true', required, 'Custom retry policy implementation class name')
+    +java-class('Class name:', retry + '.className', '"customClassName"', 'true', required, 'Custom retry policy implementation class name', required)

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade
index 7fd78bf..9fa9fc9 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade
@@ -46,7 +46,7 @@ include /app/helpers/jade/mixins.jade
                         .group-content(ng-show='#{loadBalancingSpi} && #{loadBalancingSpi}.length > 0' ng-repeat='model in #{loadBalancingSpi} track by $index')
                             hr(ng-if='$index != 0')
                             .settings-row
-                                +dropdown-required('Load balancing:', 'model.kind', '"loadBalancingKind" + $index', 'true', 'true', 'Choose load balancing SPI', '[\
+                                +dropdown-required-autofocus('Load balancing:', 'model.kind', '"loadBalancingKind" + $index', 'true', 'true', 'Choose load balancing SPI', '[\
                                         {value: "RoundRobin", label: "Round-robin"},\
                                         {value: "Adaptive", label: "Adaptive"},\
                                         {value: "WeightedRandom", label: "Random"},\
@@ -78,27 +78,30 @@ include /app/helpers/jade/mixins.jade
                                         <li>Default - Default load probing implementation</li>\
                                     </ul>')
                             .settings-row(ng-show='model.kind === "Adaptive" && model.Adaptive.loadProbe.kind')
-                                .panel-details
-                                    .details-row(ng-show='model.Adaptive.loadProbe.kind === "Job"')
+                                .panel-details(ng-show='model.Adaptive.loadProbe.kind === "Job"')
+                                    .details-row
                                         +checkbox('Use average', 'model.Adaptive.loadProbe.Job.useAverage', '"loadBalancingAdaptiveJobUseAverage" + $index', 'Use average CPU load vs. current')
-                                    .details-row(ng-show='model.Adaptive.loadProbe.kind === "CPU"')
+                                .panel-details(ng-show='model.Adaptive.loadProbe.kind === "CPU"')
+                                    .details-row
                                         +checkbox('Use average', 'model.Adaptive.loadProbe.CPU.useAverage', '"loadBalancingAdaptiveCPUUseAverage" + $index', 'Use average CPU load vs. current')
-                                    .details-row(ng-show='model.Adaptive.loadProbe.kind === "CPU"')
+                                    .details-row
                                         +checkbox('Use processors', 'model.Adaptive.loadProbe.CPU.useProcessors', '"loadBalancingAdaptiveCPUUseProcessors" + $index', "divide each node's CPU load by the number of processors on that node")
-                                    .details-row(ng-show='model.Adaptive.loadProbe.kind === "CPU"')
+                                    .details-row
                                         +number-min-max-step('Processor coefficient:', 'model.Adaptive.loadProbe.CPU.processorCoefficient',
                                             '"loadBalancingAdaptiveCPUProcessorCoefficient" + $index', 'true', '1', '0.001', '1', '0.05', 'Coefficient of every CPU')
-                                    .details-row(ng-show='model.Adaptive.loadProbe.kind === "ProcessingTime"')
+                                .panel-details(ng-show='model.Adaptive.loadProbe.kind === "ProcessingTime"')
+                                    .details-row
                                         +checkbox('Use average', 'model.Adaptive.loadProbe.ProcessingTime.useAverage', '"loadBalancingAdaptiveJobUseAverage" + $index', 'Use average execution time vs. current')
-                                    .details-row(ng-show=loadProbeCustom)
+                                .panel-details(ng-show=loadProbeCustom)
+                                    .details-row
                                         +java-class('Load brobe implementation:', 'model.Adaptive.loadProbe.Custom.className', '"loadBalancingAdaptiveJobUseClass" + $index', 'true', loadProbeCustom,
-                                            'Custom load balancing SPI implementation class name.')
+                                            'Custom load balancing SPI implementation class name.', loadProbeCustom)
                             .settings-row(ng-show='model.kind === "WeightedRandom"')
                                 +number('Node weight:', 'model.WeightedRandom.nodeWeight', '"loadBalancingWRNodeWeight" + $index', 'true', 10, '1', 'Weight of node')
                             .settings-row(ng-show='model.kind === "WeightedRandom"')
                                 +checkbox('Use weights', 'model.WeightedRandom.useWeights', '"loadBalancingWRUseWeights" + $index', 'Node weights should be checked when doing random load balancing')
                             .settings-row(ng-show=loadBalancingCustom)
                                 +java-class('Load balancing SPI implementation:', 'model.Custom.className', '"loadBalancingClass" + $index', 'true', loadBalancingCustom,
-                                    'Custom load balancing SPI implementation class name.')
+                                    'Custom load balancing SPI implementation class name.', loadBalancingCustom)
             .col-sm-6
                 +preview-xml-java(model, 'clusterLoadBalancing')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade
index 385d647..87d2b7d 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade
@@ -22,4 +22,4 @@ include /app/helpers/jade/mixins.jade
 
 div
     .details-row
-        +java-class('Class:', model + '.class', '"customLogger"', 'true', required, 'Logger implementation class name')
+        +java-class('Class:', model + '.class', '"customLogger"', 'true', required, 'Logger implementation class name', required)

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade
index 85ec073..fbd979c 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade
@@ -72,7 +72,7 @@ include /app/helpers/jade/mixins.jade
                                     label.col-xs-12.col-sm-12.col-md-12
                                         .indexField
                                             | {{ $index+1 }})
-                                        +table-remove-conditional-button(trust, enabled, 'Remove trust manager')
+                                        +table-remove-conditional-button(trust, enabled, 'Remove trust manager', 'model')
                                         span(ng-hide='field.edit')
                                             a.labelFormField(ng-click='#{enabled} && (field.edit = true) && (#{model} = model)') {{ model }}
                                         span(ng-if='field.edit')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/summary/summary-zipper.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/summary/summary-zipper.service.js b/modules/web-console/frontend/app/modules/states/configuration/summary/summary-zipper.service.js
new file mode 100644
index 0000000..08cfa71
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/states/configuration/summary/summary-zipper.service.js
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+import Worker from 'worker?inline=true!./summary.worker';
+
+export default ['$q', function($q) {
+    return function({ cluster, data }) {
+        const defer = $q.defer();
+        const worker = new Worker();
+
+        worker.postMessage({ cluster, data });
+
+        worker.onmessage = (e) => {
+            defer.resolve(e.data);
+        };
+
+        worker.onerror = (err) => {
+            defer.reject(err);
+        };
+
+        return defer.promise;
+    };
+}];

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js
index d739c43..cfc6df9 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js
+++ b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js
@@ -16,15 +16,19 @@
  */
 
 import _ from 'lodash';
-import JSZip from 'jszip';
 import saver from 'file-saver';
 
+const escapeFileName = (name) => name.replace(/[\\\/*\"\[\],\.:;|=<>?]/g, '-').replace(/ /g, '_');
+
 export default [
-    '$rootScope', '$scope', '$http', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteLoading', '$filter', 'IgniteConfigurationResource', 'JavaTypes', 'IgniteVersion', 'IgniteConfigurationGenerator', 'SpringTransformer', 'JavaTransformer', 'GeneratorDocker', 'GeneratorPom', 'IgnitePropertiesGenerator', 'IgniteReadmeGenerator', 'IgniteFormUtils',
-    function($root, $scope, $http, LegacyUtils, Messages, Loading, $filter, Resource, JavaTypes, Version, generator, spring, java, docker, pom, propsGenerator, readme, FormUtils) {
+    '$rootScope', '$scope', '$http', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteLoading', '$filter', 'IgniteConfigurationResource', 'JavaTypes', 'IgniteVersion', 'IgniteConfigurationGenerator', 'SpringTransformer', 'JavaTransformer', 'IgniteDockerGenerator', 'IgniteMavenGenerator', 'IgnitePropertiesGenerator', 'IgniteReadmeGenerator', 'IgniteFormUtils', 'IgniteSummaryZipper',
+    function($root, $scope, $http, LegacyUtils, Messages, Loading, $filter, Resource, JavaTypes, Version, generator, spring, java, docker, pom, propsGenerator, readme, FormUtils, SummaryZipper) {
         const ctrl = this;
 
-        $scope.ui = { ready: false };
+        $scope.ui = {
+            isSafari: !!(/constructor/i.test(window.HTMLElement) || window.safari),
+            ready: false
+        };
 
         Loading.start('summaryPage');
 
@@ -223,10 +227,6 @@ export default [
             return false;
         }
 
-        function escapeFileName(name) {
-            return name.replace(/[\\\/*\"\[\],\.:;|=<>?]/g, '-').replace(/ /g, '_');
-        }
-
         $scope.selectItem = (cluster) => {
             delete ctrl.cluster;
 
@@ -297,84 +297,19 @@ export default [
 
         // TODO IGNITE-2114: implemented as independent logic for download.
         $scope.downloadConfiguration = function() {
-            const cluster = $scope.cluster;
-
-            const zip = new JSZip();
-
-            if (!ctrl.data)
-                ctrl.data = {};
-
-            if (!ctrl.data.docker)
-                ctrl.data.docker = docker.generate(cluster, 'latest');
-
-            zip.file('Dockerfile', ctrl.data.docker);
-            zip.file('.dockerignore', docker.ignoreFile());
-
-            const cfg = generator.igniteConfiguration(cluster, false);
-            const clientCfg = generator.igniteConfiguration(cluster, true);
-            const clientNearCaches = _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled'));
-
-            const secProps = propsGenerator.generate(cfg);
-
-            if (secProps)
-                zip.file('src/main/resources/secret.properties', secProps);
-
-            const srcPath = 'src/main/java';
-            const resourcesPath = 'src/main/resources';
-
-            const serverXml = `${escapeFileName(cluster.name)}-server.xml`;
-            const clientXml = `${escapeFileName(cluster.name)}-client.xml`;
-
-            const metaPath = `${resourcesPath}/META-INF`;
-
-            zip.file(`${metaPath}/${serverXml}`, spring.igniteConfiguration(cfg).asString());
-            zip.file(`${metaPath}/${clientXml}`, spring.igniteConfiguration(clientCfg, clientNearCaches).asString());
-
-            const cfgPath = `${srcPath}/config`;
-
-            zip.file(`${cfgPath}/ServerConfigurationFactory.java`, java.igniteConfiguration(cfg, 'config', 'ServerConfigurationFactory').asString());
-            zip.file(`${cfgPath}/ClientConfigurationFactory.java`, java.igniteConfiguration(clientCfg, 'config', 'ClientConfigurationFactory', clientNearCaches).asString());
-
-            if (java.isDemoConfigured(cluster, $root.IgniteDemoMode)) {
-                zip.file(`${srcPath}/demo/DemoStartup.java`, java.nodeStartup(cluster, 'demo.DemoStartup',
-                    'ServerConfigurationFactory.createConfiguration()', 'config.ServerConfigurationFactory'));
-            }
-
-            // Generate loader for caches with configured store.
-            const cachesToLoad = _.filter(cluster.caches, (cache) => _.nonNil(cache.cacheStoreFactory));
-
-            if (_.nonEmpty(cachesToLoad))
-                zip.file(`${srcPath}/load/LoadCaches.java`, java.loadCaches(cachesToLoad, 'load', 'LoadCaches', `"${clientXml}"`));
-
-            const startupPath = `${srcPath}/startup`;
-
-            zip.file(`${startupPath}/ServerNodeSpringStartup.java`, java.nodeStartup(cluster, 'startup.ServerNodeSpringStartup', `"${serverXml}"`));
-            zip.file(`${startupPath}/ClientNodeSpringStartup.java`, java.nodeStartup(cluster, 'startup.ClientNodeSpringStartup', `"${clientXml}"`));
-
-            zip.file(`${startupPath}/ServerNodeCodeStartup.java`, java.nodeStartup(cluster, 'startup.ServerNodeCodeStartup',
-                'ServerConfigurationFactory.createConfiguration()', 'config.ServerConfigurationFactory'));
-            zip.file(`${startupPath}/ClientNodeCodeStartup.java`, java.nodeStartup(cluster, 'startup.ClientNodeCodeStartup',
-                'ClientConfigurationFactory.createConfiguration()', 'config.ClientConfigurationFactory', clientNearCaches));
-
-            zip.file('pom.xml', pom.generate(cluster, Version.productVersion().ignite).asString());
-
-            zip.file('README.txt', readme.generate());
-            zip.file('jdbc-drivers/README.txt', readme.generateJDBC());
-
-            if (_.isEmpty(ctrl.data.pojos))
-                ctrl.data.pojos = java.pojos(cluster.caches);
-
-            for (const pojo of ctrl.data.pojos) {
-                if (pojo.keyClass && JavaTypes.nonBuiltInClass(pojo.keyType))
-                    zip.file(`${srcPath}/${pojo.keyType.replace(/\./g, '/')}.java`, pojo.keyClass);
+            if ($scope.isPrepareDownloading)
+                return;
 
-                zip.file(`${srcPath}/${pojo.valueType.replace(/\./g, '/')}.java`, pojo.valueClass);
-            }
+            const cluster = $scope.cluster;
 
-            $generatorOptional.optionalContent(zip, cluster);
+            $scope.isPrepareDownloading = true;
 
-            zip.generateAsync({type: 'blob', compression: 'DEFLATE', mimeType: 'application/octet-stream'})
-                .then((blob) => saver.saveAs(blob, escapeFileName(cluster.name) + '-project.zip'));
+            return new SummaryZipper({ cluster, data: ctrl.data || {}, IgniteDemoMode: $root.IgniteDemoMode })
+                .then((data) => {
+                    saver.saveAs(data, escapeFileName(cluster.name) + '-project.zip');
+                })
+                .catch((err) => Messages.showError('Failed to generate project files. ' + err.message))
+                .then(() => $scope.isPrepareDownloading = false);
         };
 
         /**
@@ -393,7 +328,7 @@ export default [
             const dialects = $scope.dialects;
 
             if (dialects.Oracle)
-                window.open('http://www.oracle.com/technetwork/apps-tech/jdbc-112010-090769.html');
+                window.open('http://www.oracle.com/technetwork/database/features/jdbc/default-2280470.html');
 
             if (dialects.DB2)
                 window.open('http://www-01.ibm.com/support/docview.wss?uid=swg21363866');

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js
new file mode 100644
index 0000000..6b24001
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import JSZip from 'jszip';
+
+import IgniteVersion from 'app/modules/configuration/Version.service';
+
+import MavenGenerator from 'app/modules/configuration/generator/Maven.service';
+import DockerGenerator from 'app/modules/configuration/generator/Docker.service';
+import ReadmeGenerator from 'app/modules/configuration/generator/Readme.service';
+import PropertiesGenerator from 'app/modules/configuration/generator/Properties.service';
+import ConfigurationGenerator from 'app/modules/configuration/generator/ConfigurationGenerator';
+
+import JavaTransformer from 'app/modules/configuration/generator/JavaTransformer.service';
+import SpringTransformer from 'app/modules/configuration/generator/SpringTransformer.service';
+
+const Version = new IgniteVersion();
+
+const maven = new MavenGenerator();
+const docker = new DockerGenerator();
+const readme = new ReadmeGenerator();
+const properties = new PropertiesGenerator();
+
+const java = new JavaTransformer[0]();
+const spring = new SpringTransformer[0]();
+
+const generator = new ConfigurationGenerator[0]();
+
+const escapeFileName = (name) => name.replace(/[\\\/*\"\[\],\.:;|=<>?]/g, '-').replace(/ /g, '_');
+
+// eslint-disable-next-line no-undef
+onmessage = function(e) {
+    const {cluster, data, demo} = e.data;
+
+    const zip = new JSZip();
+
+    if (!data.docker)
+        data.docker = docker.generate(cluster, 'latest');
+
+    zip.file('Dockerfile', data.docker);
+    zip.file('.dockerignore', docker.ignoreFile());
+
+    const cfg = generator.igniteConfiguration(cluster, false);
+    const clientCfg = generator.igniteConfiguration(cluster, true);
+    const clientNearCaches = _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled'));
+
+    const secProps = properties.generate(cfg);
+
+    if (secProps)
+        zip.file('src/main/resources/secret.properties', secProps);
+
+    const srcPath = 'src/main/java';
+    const resourcesPath = 'src/main/resources';
+
+    const serverXml = `${escapeFileName(cluster.name)}-server.xml`;
+    const clientXml = `${escapeFileName(cluster.name)}-client.xml`;
+
+    const metaPath = `${resourcesPath}/META-INF`;
+
+    zip.file(`${metaPath}/${serverXml}`, spring.igniteConfiguration(cfg).asString());
+    zip.file(`${metaPath}/${clientXml}`, spring.igniteConfiguration(clientCfg, clientNearCaches).asString());
+
+    const cfgPath = `${srcPath}/config`;
+
+    zip.file(`${cfgPath}/ServerConfigurationFactory.java`, java.igniteConfiguration(cfg, 'config', 'ServerConfigurationFactory').asString());
+    zip.file(`${cfgPath}/ClientConfigurationFactory.java`, java.igniteConfiguration(clientCfg, 'config', 'ClientConfigurationFactory', clientNearCaches).asString());
+
+    if (java.isDemoConfigured(cluster, demo)) {
+        zip.file(`${srcPath}/demo/DemoStartup.java`, java.nodeStartup(cluster, 'demo.DemoStartup',
+            'ServerConfigurationFactory.createConfiguration()', 'config.ServerConfigurationFactory'));
+    }
+
+    // Generate loader for caches with configured store.
+    const cachesToLoad = _.filter(cluster.caches, (cache) => _.nonNil(cache.cacheStoreFactory));
+
+    if (_.nonEmpty(cachesToLoad))
+        zip.file(`${srcPath}/load/LoadCaches.java`, java.loadCaches(cachesToLoad, 'load', 'LoadCaches', `"${clientXml}"`));
+
+    const startupPath = `${srcPath}/startup`;
+
+    zip.file(`${startupPath}/ServerNodeSpringStartup.java`, java.nodeStartup(cluster, 'startup.ServerNodeSpringStartup', `"${serverXml}"`));
+    zip.file(`${startupPath}/ClientNodeSpringStartup.java`, java.nodeStartup(cluster, 'startup.ClientNodeSpringStartup', `"${clientXml}"`));
+
+    zip.file(`${startupPath}/ServerNodeCodeStartup.java`, java.nodeStartup(cluster, 'startup.ServerNodeCodeStartup',
+        'ServerConfigurationFactory.createConfiguration()', 'config.ServerConfigurationFactory'));
+    zip.file(`${startupPath}/ClientNodeCodeStartup.java`, java.nodeStartup(cluster, 'startup.ClientNodeCodeStartup',
+        'ClientConfigurationFactory.createConfiguration()', 'config.ClientConfigurationFactory', clientNearCaches));
+
+    zip.file('pom.xml', maven.generate(cluster, Version.productVersion().ignite).asString());
+
+    zip.file('README.txt', readme.generate());
+    zip.file('jdbc-drivers/README.txt', readme.generateJDBC());
+
+    if (_.isEmpty(data.pojos))
+        data.pojos = java.pojos(cluster.caches);
+
+    for (const pojo of data.pojos) {
+        if (pojo.keyClass)
+            zip.file(`${srcPath}/${pojo.keyType.replace(/\./g, '/')}.java`, pojo.keyClass);
+
+        zip.file(`${srcPath}/${pojo.valueType.replace(/\./g, '/')}.java`, pojo.valueClass);
+    }
+
+    zip.generateAsync({
+        type: 'blob',
+        compression: 'DEFLATE',
+        mimeType: 'application/octet-stream'
+    }).then((blob) => postMessage(blob));
+};

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/user/Auth.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/user/Auth.service.js b/modules/web-console/frontend/app/modules/user/Auth.service.js
index 43e2f92..e0f905d 100644
--- a/modules/web-console/frontend/app/modules/user/Auth.service.js
+++ b/modules/web-console/frontend/app/modules/user/Auth.service.js
@@ -20,12 +20,11 @@ export default ['Auth', ['$http', '$rootScope', '$state', '$window', 'IgniteErro
         return {
             forgotPassword(userInfo) {
                 $http.post('/api/v1/password/forgot', userInfo)
-                    .success(() => $state.go('password.send'))
-                    .error((err) => ErrorPopover.show('forgot_email', Messages.errorMessage(null, err)));
+                    .then(() => $state.go('password.send'))
+                    .cacth(({data}) => ErrorPopover.show('forgot_email', Messages.errorMessage(null, data)));
             },
             auth(action, userInfo) {
                 $http.post('/api/v1/' + action, userInfo)
-                    .catch(({data}) => Promise.reject(data))
                     .then(() => {
                         if (action === 'password/forgot')
                             return;
@@ -41,16 +40,16 @@ export default ['Auth', ['$http', '$rootScope', '$state', '$window', 'IgniteErro
                                 $root.gettingStarted.tryShow();
                             });
                     })
-                    .catch((err) => ErrorPopover.show(action + '_email', Messages.errorMessage(null, err)));
+                    .catch((res) => ErrorPopover.show(action + '_email', Messages.errorMessage(null, res)));
             },
             logout() {
                 $http.post('/api/v1/logout')
-                    .success(() => {
+                    .then(() => {
                         User.clean();
 
                         $window.open($state.href('signin'), '_self');
                     })
-                    .error(Messages.showError);
+                    .catch(Messages.showError);
             }
         };
     }]];

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/services/JavaTypes.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/services/JavaTypes.service.js b/modules/web-console/frontend/app/services/JavaTypes.service.js
index 679914f..944fea5 100644
--- a/modules/web-console/frontend/app/services/JavaTypes.service.js
+++ b/modules/web-console/frontend/app/services/JavaTypes.service.js
@@ -40,7 +40,7 @@ const VALID_UUID = /^[0-9a-f]{8}-[0-9a-f]{4}-[1-5][0-9a-f]{3}-[89ab][0-9a-f]{3}-
  * Utility service for various check on java types.
  */
 export default class JavaTypes {
-    static $inject = ['igniteClusterDefaults', 'igniteCacheDefaults', 'igniteIgfsDefaults'];
+    static $inject = ['IgniteClusterDefaults', 'IgniteCacheDefaults', 'IgniteIGFSDefaults'];
 
     constructor(clusterDflts, cacheDflts, igfsDflts) {
         this.enumClasses = _.uniq(this._enumClassesAcc(_.merge(clusterDflts, cacheDflts, igfsDflts), []));
@@ -101,14 +101,9 @@ export default class JavaTypes {
      * @return {String} Class name.
      */
     shortClassName(clsName) {
-        if (this.isJavaPrimitive(clsName))
-            return clsName;
+        const dotIdx = clsName.lastIndexOf('.');
 
-        const fullClsName = this.fullClassName(clsName);
-
-        const dotIdx = fullClsName.lastIndexOf('.');
-
-        return dotIdx > 0 ? fullClsName.substr(dotIdx + 1) : fullClsName;
+        return dotIdx > 0 ? clsName.substr(dotIdx + 1) : clsName;
     }
 
     /**
@@ -163,7 +158,7 @@ export default class JavaTypes {
      * @param {String} clsName Class name to check.
      * @returns {boolean} 'true' if given class name is java primitive.
      */
-    isJavaPrimitive(clsName) {
+    isPrimitive(clsName) {
         return _.includes(JAVA_PRIMITIVES, clsName);
     }
 


[15/50] [abbrv] ignite git commit: TcpDiscoverySpi: need check result of sock.getSendBufferSize().

Posted by yz...@apache.org.
TcpDiscoverySpi: need check result of sock.getSendBufferSize().


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 0a336445367fb2cec202c431a8614c185da9a3ca
Parents: 854d1a5
Author: sboikov <sb...@gridgain.com>
Authored: Wed Dec 21 13:28:25 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Dec 21 13:28:25 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/spi/discovery/tcp/ServerImpl.java   |  2 +-
 .../ignite/spi/discovery/tcp/TcpDiscoverySpi.java     | 14 +++++++++++++-
 2 files changed, 14 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0a336445/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 50fa3bd..7cc35e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -2771,7 +2771,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                                 sock = spi.openSocket(addr, timeoutHelper);
 
-                                out = new BufferedOutputStream(sock.getOutputStream(), sock.getSendBufferSize());
+                                out = spi.socketStream(sock);
 
                                 openSock = true;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0a336445/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index 45933e1..db85cc4 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -1246,6 +1246,18 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     }
 
     /**
+     * @param sock Socket.
+     * @return Buffered stream wrapping socket stream.
+     * @throws IOException If failed.
+     */
+    final BufferedOutputStream socketStream(Socket sock) throws IOException {
+        int bufSize = sock.getSendBufferSize();
+
+        return bufSize > 0 ? new BufferedOutputStream(sock.getOutputStream(), bufSize) :
+            new BufferedOutputStream(sock.getOutputStream());
+    }
+
+    /**
      * Connects to remote address sending {@code U.IGNITE_HEADER} when connection is established.
      *
      * @param sock Socket bound to a local host address.
@@ -1351,7 +1363,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
      */
     protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg, long timeout) throws IOException,
         IgniteCheckedException {
-        writeToSocket(sock, new BufferedOutputStream(sock.getOutputStream(), sock.getSendBufferSize()), msg, timeout);
+        writeToSocket(sock, socketStream(sock), msg, timeout);
     }
 
     /**


[45/50] [abbrv] ignite git commit: 2.0.0-SNAPSHOT

Posted by yz...@apache.org.
2.0.0-SNAPSHOT


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 10b2b97a38cf52e480785ebb041f51e562666ad6
Parents: 32d2629
Author: Ignite Teamcity <ig...@apache.org>
Authored: Tue Dec 27 14:04:05 2016 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Tue Dec 27 14:04:05 2016 +0300

----------------------------------------------------------------------
 modules/platforms/cpp/configure.ac                               | 2 +-
 modules/platforms/cpp/configure.acrel                            | 2 +-
 modules/platforms/cpp/examples/configure.ac                      | 2 +-
 modules/platforms/cpp/odbc/install/ignite-odbc-amd64.wxs         | 2 +-
 modules/platforms/cpp/odbc/install/ignite-odbc-x86.wxs           | 2 +-
 .../dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs | 4 ++--
 .../dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs       | 4 ++--
 .../dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs   | 4 ++--
 .../Apache.Ignite.Core.Tests.NuGet/Properties/AssemblyInfo.cs    | 4 ++--
 .../Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs  | 4 ++--
 .../dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs   | 4 ++--
 .../dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs         | 4 ++--
 .../Properties/AssemblyInfo.cs                                   | 4 ++--
 .../Apache.Ignite.EntityFramework/Properties/AssemblyInfo.cs     | 4 ++--
 .../dotnet/Apache.Ignite.Linq/Properties/AssemblyInfo.cs         | 4 ++--
 .../dotnet/Apache.Ignite.Log4Net/Properties/AssemblyInfo.cs      | 4 ++--
 .../dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs         | 4 ++--
 .../platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs    | 4 ++--
 .../examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs   | 4 ++--
 .../Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs         | 4 ++--
 20 files changed, 35 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/modules/platforms/cpp/configure.ac
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/configure.ac b/modules/platforms/cpp/configure.ac
index 0c1f731..3c98225 100644
--- a/modules/platforms/cpp/configure.ac
+++ b/modules/platforms/cpp/configure.ac
@@ -19,7 +19,7 @@
 # Process this file with autoconf to produce a configure script.
 
 AC_PREREQ([2.69])
-AC_INIT([Apache Ignite C++], [2.0.0.16263], [dev@ignite.apache.org], [ignite], [ignite.apache.org])
+AC_INIT([Apache Ignite C++], [2.0.0.16694], [dev@ignite.apache.org], [ignite], [ignite.apache.org])
 
 AC_CANONICAL_HOST
 AC_CONFIG_MACRO_DIR([m4])

http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/modules/platforms/cpp/configure.acrel
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/configure.acrel b/modules/platforms/cpp/configure.acrel
index 9a0d87f..f73b57c 100644
--- a/modules/platforms/cpp/configure.acrel
+++ b/modules/platforms/cpp/configure.acrel
@@ -19,7 +19,7 @@
 # Process this file with autoconf to produce a configure script.
 
 AC_PREREQ([2.69])
-AC_INIT([Apache Ignite C++], [2.0.0.16263], [dev@ignite.apache.org], [ignite], [ignite.apache.org])
+AC_INIT([Apache Ignite C++], [2.0.0.16694], [dev@ignite.apache.org], [ignite], [ignite.apache.org])
 
 AC_CANONICAL_HOST
 AC_CONFIG_MACRO_DIR([m4])

http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/modules/platforms/cpp/examples/configure.ac
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/configure.ac b/modules/platforms/cpp/examples/configure.ac
index d167cf0..e15d186 100644
--- a/modules/platforms/cpp/examples/configure.ac
+++ b/modules/platforms/cpp/examples/configure.ac
@@ -19,7 +19,7 @@
 # Process this file with autoconf to produce a configure script.
 
 AC_PREREQ([2.69])
-AC_INIT([Apache Ignite C++ Examples], [2.0.0.16263], [dev@ignite.apache.org], [ignite-examples], [ignite.apache.org])
+AC_INIT([Apache Ignite C++ Examples], [2.0.0.16694], [dev@ignite.apache.org], [ignite-examples], [ignite.apache.org])
 
 AC_CANONICAL_HOST
 AC_CONFIG_MACRO_DIR([m4])

http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/modules/platforms/cpp/odbc/install/ignite-odbc-amd64.wxs
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/install/ignite-odbc-amd64.wxs b/modules/platforms/cpp/odbc/install/ignite-odbc-amd64.wxs
index 6ec1c31..eadbe88 100644
--- a/modules/platforms/cpp/odbc/install/ignite-odbc-amd64.wxs
+++ b/modules/platforms/cpp/odbc/install/ignite-odbc-amd64.wxs
@@ -21,7 +21,7 @@
 	<Product Name='Apache Ignite ODBC 64-bit Driver' Manufacturer='The Apache Software Foundation'
 		Id='F3E308E4-910C-4AF5-82DE-2ACF4D64830E' 
 		UpgradeCode='1D7AEFDF-6CD2-4FB5-88F2-811A89832D6D'
-		Language='1033' Codepage='1252' Version='2.0.0.16263'>
+		Language='1033' Codepage='1252' Version='2.0.0.16694'>
 		
 		<Package Id='*' Keywords='Installer' Description="Apache Ignite ODBC 64-bit Driver Installer"
 			Comments='Apache, Apache Ignite, the Apache feather and the Apache Ignite logo are trademarks of The Apache Software Foundation.'

http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/modules/platforms/cpp/odbc/install/ignite-odbc-x86.wxs
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/install/ignite-odbc-x86.wxs b/modules/platforms/cpp/odbc/install/ignite-odbc-x86.wxs
index c1aeca6..b50b5a3 100644
--- a/modules/platforms/cpp/odbc/install/ignite-odbc-x86.wxs
+++ b/modules/platforms/cpp/odbc/install/ignite-odbc-x86.wxs
@@ -21,7 +21,7 @@
 	<Product Name='Apache Ignite ODBC 32-bit Driver' Manufacturer='The Apache Software Foundation'
 		Id='D39CBABA-1E21-4701-AA5C-91EDA07B383B' 
 		UpgradeCode='743902A4-365C-424E-B226-5B2898A3941E'
-		Language='1033' Codepage='1252' Version='2.0.0.16263'>
+		Language='1033' Codepage='1252' Version='2.0.0.16694'>
 		
 		<Package Id='*' Keywords='Installer' Description="Apache Ignite ODBC 32-bit Driver Installer"
 			Comments='Apache, Apache Ignite, the Apache feather and the Apache Ignite logo are trademarks of The Apache Software Foundation.'

http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs
index d152218..1bca0e8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs
@@ -35,8 +35,8 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("18ea4c71-a11d-4ab1-8042-418f7559d84f")]
 
-[assembly: AssemblyVersion("2.0.0.16263")]
-[assembly: AssemblyFileVersion("2.0.0.16263")]
+[assembly: AssemblyVersion("2.0.0.16694")]
+[assembly: AssemblyFileVersion("2.0.0.16694")]
 [assembly: AssemblyInformationalVersion("2.0.0")]
 
 [assembly: CLSCompliant(true)]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs
index ce5fcbb..0926a46 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs
@@ -33,8 +33,8 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("13ea96fc-cc83-4164-a7c0-4f30ed797460")]
 
-[assembly: AssemblyVersion("2.0.0.16263")]
-[assembly: AssemblyFileVersion("2.0.0.16263")]
+[assembly: AssemblyVersion("2.0.0.16694")]
+[assembly: AssemblyFileVersion("2.0.0.16694")]
 [assembly: AssemblyInformationalVersion("2.0.0")]
 
 [assembly: CLSCompliant(true)]

http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs
index 5685f78..7da0a1d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs
@@ -31,6 +31,6 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("8fae8395-7e91-411a-a78f-44d6d3fed0fc")]
 
-[assembly: AssemblyVersion("2.0.0.16263")]
-[assembly: AssemblyFileVersion("2.0.0.16263")]
+[assembly: AssemblyVersion("2.0.0.16694")]
+[assembly: AssemblyFileVersion("2.0.0.16694")]
 [assembly: AssemblyInformationalVersion("2.0.0")]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Properties/AssemblyInfo.cs
index cfb16a8..b7d8e09 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Properties/AssemblyInfo.cs
@@ -30,6 +30,6 @@ using System.Runtime.InteropServices;
 [assembly: ComVisible(false)]
 [assembly: Guid("134707f6-155d-47f6-9eb2-c67abbf3c009")]
 
-[assembly: AssemblyVersion("2.0.0.16263")]
-[assembly: AssemblyFileVersion("2.0.0.16263")]
+[assembly: AssemblyVersion("2.0.0.16694")]
+[assembly: AssemblyFileVersion("2.0.0.16694")]
 [assembly: AssemblyInformationalVersion("2.0.0")]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
index 9c55a10..26caa82 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
@@ -45,6 +45,6 @@ using System.Runtime.InteropServices;
 // You can specify all the values or you can default the Build and Revision Numbers
 // by using the '*' as shown below:
 // [assembly: AssemblyVersion("1.0.*")]
-[assembly: AssemblyVersion("2.0.0.16263")]
-[assembly: AssemblyFileVersion("2.0.0.16263")]
+[assembly: AssemblyVersion("2.0.0.16694")]
+[assembly: AssemblyFileVersion("2.0.0.16694")]
 [assembly: AssemblyInformationalVersion("2.0.0")]

http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
index 8010712..1fc6c59 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
@@ -31,6 +31,6 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("de8dd5cc-7c7f-4a09-80d5-7086d9416a7b")]
 
-[assembly: AssemblyVersion("2.0.0.16263")]
-[assembly: AssemblyFileVersion("2.0.0.16263")]
+[assembly: AssemblyVersion("2.0.0.16694")]
+[assembly: AssemblyFileVersion("2.0.0.16694")]
 [assembly: AssemblyInformationalVersion("2.0.0")]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
index 666c2fc..c0462db 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
@@ -33,8 +33,8 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("97db45a8-f922-456a-a819-7b3c6e5e03ba")]
 
-[assembly: AssemblyVersion("2.0.0.16263")]
-[assembly: AssemblyFileVersion("2.0.0.16263")]
+[assembly: AssemblyVersion("2.0.0.16694")]
+[assembly: AssemblyFileVersion("2.0.0.16694")]
 [assembly: AssemblyInformationalVersion("2.0.0")]
 
 [assembly: CLSCompliant(true)]

http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/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
index c7fada9..8aeec58 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Properties/AssemblyInfo.cs
@@ -32,8 +32,8 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("cda5700e-78f3-4a9e-a9b0-704cbe94651c")]
 
-[assembly: AssemblyVersion("2.0.0.16263")]
-[assembly: AssemblyFileVersion("2.0.0.16263")]
+[assembly: AssemblyVersion("2.0.0.16694")]
+[assembly: AssemblyFileVersion("2.0.0.16694")]
 [assembly: AssemblyInformationalVersion("2.0.0")]
 
 [assembly: CLSCompliant(true)]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/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
index 3454f61..5346eba 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Properties/AssemblyInfo.cs
@@ -32,8 +32,8 @@ using System.Runtime.InteropServices;
 [assembly: ComVisible(false)]
 [assembly: Guid("c558518a-c1a0-4224-aaa9-a8688474b4dc")]
 
-[assembly: AssemblyVersion("2.0.0.16263")]
-[assembly: AssemblyFileVersion("2.0.0.16263")]
+[assembly: AssemblyVersion("2.0.0.16694")]
+[assembly: AssemblyFileVersion("2.0.0.16694")]
 [assembly: AssemblyInformationalVersion("2.0.0")]
 
 [assembly: CLSCompliant(true)]

http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/modules/platforms/dotnet/Apache.Ignite.Linq/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Properties/AssemblyInfo.cs
index 29b1620..a1dae5e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Properties/AssemblyInfo.cs
@@ -33,8 +33,8 @@ using System.Runtime.InteropServices;
 // The following GUID is for the ID of the typelib if this project is exposed to COM
 [assembly: Guid("5b571661-17f4-4f29-8c7d-0edb38ca9b55")]
 
-[assembly: AssemblyVersion("2.0.0.16263")]
-[assembly: AssemblyFileVersion("2.0.0.16263")]
+[assembly: AssemblyVersion("2.0.0.16694")]
+[assembly: AssemblyFileVersion("2.0.0.16694")]
 [assembly: AssemblyInformationalVersion("2.0.0")]
 
 [assembly: CLSCompliant(true)]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/modules/platforms/dotnet/Apache.Ignite.Log4Net/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Log4Net/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Log4Net/Properties/AssemblyInfo.cs
index 9b8eedc..cc37917 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Log4Net/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Log4Net/Properties/AssemblyInfo.cs
@@ -33,8 +33,8 @@ using System.Runtime.InteropServices;
 // The following GUID is for the ID of the typelib if this project is exposed to COM
 [assembly: Guid("6f82d669-382e-4435-8092-68c4440146d8")]
 
-[assembly: AssemblyVersion("2.0.0.16263")]
-[assembly: AssemblyFileVersion("2.0.0.16263")]
+[assembly: AssemblyVersion("2.0.0.16694")]
+[assembly: AssemblyFileVersion("2.0.0.16694")]
 [assembly: AssemblyInformationalVersion("2.0.0")]
 
 [assembly: CLSCompliant(true)]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs
index 943065d..2f0d89f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs
@@ -33,8 +33,8 @@ using System.Runtime.InteropServices;
 // The following GUID is for the ID of the typelib if this project is exposed to COM
 [assembly: Guid("c6b58e4a-a2e9-4554-ad02-68ce6da5cfb7")]
 
-[assembly: AssemblyVersion("2.0.0.16263")]
-[assembly: AssemblyFileVersion("2.0.0.16263")]
+[assembly: AssemblyVersion("2.0.0.16694")]
+[assembly: AssemblyFileVersion("2.0.0.16694")]
 [assembly: AssemblyInformationalVersion("2.0.0")]
 
 [assembly: CLSCompliant(true)]

http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
index d8a8578..97a78d5 100644
--- a/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
@@ -31,6 +31,6 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("0f9702ec-da7d-4ce5-b4b7-73310c885355")]
 
-[assembly: AssemblyVersion("2.0.0.16263")]
-[assembly: AssemblyFileVersion("2.0.0.16263")]
+[assembly: AssemblyVersion("2.0.0.16694")]
+[assembly: AssemblyFileVersion("2.0.0.16694")]
 [assembly: AssemblyInformationalVersion("2.0.0")]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs
index ed09c6d..a75ee0e 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs
@@ -31,6 +31,6 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("41a0cb95-3435-4c78-b867-900b28e2c9ee")]
 
-[assembly: AssemblyVersion("2.0.0.16263")]
-[assembly: AssemblyFileVersion("2.0.0.16263")]
+[assembly: AssemblyVersion("2.0.0.16694")]
+[assembly: AssemblyFileVersion("2.0.0.16694")]
 [assembly: AssemblyInformationalVersion("2.0.0")]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/10b2b97a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs
index 1dc3bef..cc78a5c 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs
@@ -31,6 +31,6 @@ using System.Runtime.InteropServices;
 
 [assembly: Guid("ce65ec7c-d3cf-41ad-8f45-f90d5af68d77")]
 
-[assembly: AssemblyVersion("2.0.0.16263")]
-[assembly: AssemblyFileVersion("2.0.0.16263")]
+[assembly: AssemblyVersion("2.0.0.16694")]
+[assembly: AssemblyFileVersion("2.0.0.16694")]
 [assembly: AssemblyInformationalVersion("2.0.0")]
\ No newline at end of file


[14/50] [abbrv] ignite git commit: TcpCommunicationSpi: fixed typo in warning.

Posted by yz...@apache.org.
TcpCommunicationSpi: fixed typo in warning.


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

Branch: refs/heads/ignite-comm-balance-master
Commit: ce46c10ea767254a60b80abecf45be8a2d9921b4
Parents: e8fd165
Author: sboikov <sb...@gridgain.com>
Authored: Wed Dec 21 13:09:26 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Dec 21 13:09:26 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ce46c10e/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 0c90414..c35b5ef 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -1872,7 +1872,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         if (msgQueueLimit == 0)
             U.quietAndWarn(log, "Message queue limit is set to 0 which may lead to " +
                 "potential OOMEs when running cache operations in FULL_ASYNC or PRIMARY_SYNC modes " +
-                "due to message queues growth on sender and reciever sides.");
+                "due to message queues growth on sender and receiver sides.");
 
         registerMBean(gridName, this, TcpCommunicationSpiMBean.class);
 


[43/50] [abbrv] ignite git commit: IGNITE-4367 .NET: Fix flaky tests - fix race in ReconnectTest.TestClusterRestart

Posted by yz...@apache.org.
IGNITE-4367 .NET: Fix flaky tests - fix race in ReconnectTest.TestClusterRestart


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 300750f14eff04063e898d2083ea188ce0777e7d
Parents: 7606e66
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Dec 26 19:15:53 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Dec 26 19:15:53 2016 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.Tests/ReconnectTest.cs   | 20 +++++++++++++++++---
 1 file changed, 17 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/300750f1/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 5e9f788..253a10f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ReconnectTest.cs
@@ -70,14 +70,17 @@ namespace Apache.Ignite.Core.Tests
 
             Assert.IsNotNull(ex);
 
+            // Wait a bit for cluster restart detection.
+            Thread.Sleep(1000);
+
             // Start the server and wait for reconnect.
             Ignition.Start(serverCfg);
 
-            // Wait a bit for notifications.
-            Thread.Sleep(100);
-
             // Check reconnect task.
             Assert.IsTrue(ex.ClientReconnectTask.Result);
+            
+            // Wait a bit for notifications.
+            Thread.Sleep(100);
 
             // Check the event args.
             Assert.IsNotNull(eventArgs);
@@ -167,6 +170,17 @@ namespace Apache.Ignite.Core.Tests
                 "-J-DIGNITE_QUIET=false");
         }
 
+
+        /// <summary>
+        /// Test set up.
+        /// </summary>
+        [SetUp]
+        public void SetUp()
+        {
+            Ignition.StopAll(true);
+            IgniteProcess.KillAll();
+        }
+
         /// <summary>
         /// Test tear down.
         /// </summary>


[07/50] [abbrv] ignite git commit: ignite-4379: fixed issue after merge with latest DML changes

Posted by yz...@apache.org.
ignite-4379: fixed issue after merge with latest DML changes


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

Branch: refs/heads/ignite-comm-balance-master
Commit: e8fd1657de7847922feec3d3e58250ad116e4e47
Parents: 800579c
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 20 15:27:04 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 20 15:27:04 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/query/h2/IgniteH2Indexing.java      | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e8fd1657/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index c541185..9037943 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -801,8 +801,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         Prepared p = GridSqlQueryParser.prepared((JdbcPreparedStatement)stmt);
 
         if (!p.isQuery()) {
-            GridH2QueryContext.clearThreadLocal();
-
             SqlFieldsQuery fldsQry = new SqlFieldsQuery(qry);
 
             if (params != null)


[20/50] [abbrv] ignite git commit: IGNITE-1443 CPP: Implement cache continuous queries - minor fix

Posted by yz...@apache.org.
IGNITE-1443 CPP: Implement cache continuous queries - minor fix

This closes #1378


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 848bb66311365e5f8e3bde8310ad064d3449bf13
Parents: 1b2afbb
Author: Igor Sapego <is...@gridgain.com>
Authored: Thu Dec 22 19:14:10 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu Dec 22 19:14:10 2016 +0300

----------------------------------------------------------------------
 .../core/include/ignite/cache/query/continuous/continuous_query.h  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/848bb663/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h b/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h
index 563b11a..4bd9c46 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h
@@ -204,7 +204,7 @@ namespace ignite
                      */
                     void SetListener(Reference<event::CacheEntryEventListener<K, V>> lsnr)
                     {
-                        impl.Get()->SetListener(val);
+                        impl.Get()->SetListener(lsnr);
                     }
 
                     /**