You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2019/04/01 12:11:00 UTC

[ignite] 01/02: invokeAll

This is an automated email from the ASF dual-hosted git repository.

sboikov pushed a commit to branch ignite-invokeAll
in repository https://gitbox.apache.org/repos/asf/ignite.git

commit 5c691fd9949c63c20859b45b1f5a4dc28ad95d88
Merge: d7389ae 29b0526
Author: sboikov <sb...@apache.org>
AuthorDate: Wed Mar 27 14:39:45 2019 +0300

    invokeAll

 .gitignore                                         |    1 +
 CONTRIBUTING.md                                    |    2 +-
 assembly/docfiles/javadoc.css                      |  648 -----------
 assembly/release-apache-ignite-base.xml            |   18 +-
 bin/control.bat                                    |    2 +-
 bin/control.sh                                     |    2 +-
 bin/ignite-tf.sh                                   |    2 +-
 bin/ignite.bat                                     |    2 +-
 bin/ignite.sh                                      |   14 +-
 bin/ignitevisorcmd.bat                             |    2 +-
 bin/ignitevisorcmd.sh                              |    2 +-
 examples/README.md                                 |    8 +-
 examples/pom-standalone-lgpl.xml                   |   44 +
 examples/pom-standalone.xml                        |    1 -
 modules/clients/src/test/config/jdbc-config.xml    |    6 +
 .../internal/jdbc2/JdbcAbstractSchemaCaseTest.java |  228 ++++
 .../internal/jdbc2/JdbcMetadataSelfTest.java       |    7 +-
 ...pCacheTest.java => JdbcSchemaCaseSelfTest.java} |   20 +-
 .../tcp/redis/RedisProtocolGetAllAsArrayTest.java  |   17 +-
 .../ignite/jdbc/AbstractJdbcPojoQuerySelfTest.java |   12 -
 .../ignite/jdbc/JdbcErrorsAbstractSelfTest.java    |    2 +-
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java      |   11 +-
 .../ignite/jdbc/thin/JdbcThinAbstractSelfTest.java |    2 +-
 .../thin/JdbcThinBulkLoadAbstractSelfTest.java     |    2 +-
 .../thin/JdbcThinDataPageScanPropertySelfTest.java |  222 ++++
 .../ignite/jdbc/thin/JdbcThinMetadataSelfTest.java |    9 +-
 .../jdbc/thin/JdbcThinSchemaCaseSelfTest.java}     |   17 +-
 .../ignite/jdbc/thin/JdbcThinSchemaCaseTest.java   |  114 --
 .../thin/JdbcThinStreamingAbstractSelfTest.java    |   22 +-
 .../thin/JdbcThinStreamingResetStreamTest.java     |  142 +++
 .../java/org/apache/ignite/DataRegionMetrics.java  |   12 +
 .../apache/ignite/DataRegionMetricsProvider.java}  |   29 +-
 .../main/java/org/apache/ignite/IgniteCluster.java |   29 +-
 .../apache/ignite/IgniteJdbcThinDataSource.java    |   17 +
 .../org/apache/ignite/IgniteSystemProperties.java  |   27 +-
 .../org/apache/ignite/internal/IgniteKernal.java   |    8 +-
 .../cluster/DistributedBaselineConfiguration.java  |    2 -
 .../internal/cluster/IgniteClusterAsyncImpl.java   |   37 +-
 .../ignite/internal/cluster/IgniteClusterImpl.java |   75 +-
 .../ignite/internal/commandline/Arguments.java     |   25 +-
 .../internal/commandline/CommandHandler.java       |  407 ++-----
 .../baseline/AutoAdjustCommandArg.java}            |   35 +-
 .../commandline/baseline/BaselineArguments.java    |  165 +++
 .../commandline/baseline/BaselineCommand.java      |  103 ++
 .../internal/jdbc/thin/ConnectionProperties.java   |   13 +
 .../jdbc/thin/ConnectionPropertiesImpl.java        |   32 +-
 .../internal/jdbc/thin/JdbcThinConnection.java     |   31 +-
 .../ignite/internal/jdbc/thin/JdbcThinTcpIo.java   |   11 +-
 .../ignite/internal/jdbc/thin/JdbcThinUtils.java   |   45 +-
 .../ignite/internal/jdbc2/JdbcConnection.java      |    2 +-
 .../apache/ignite/internal/jdbc2/JdbcUtils.java    |   22 +
 .../processors/cache/CacheGroupContext.java        |    6 +-
 .../processors/cache/GridCacheAdapter.java         |   26 +-
 .../processors/cache/GridCacheContext.java         |   12 +-
 .../processors/cache/GridCacheEntryEx.java         |   11 +
 .../processors/cache/GridCacheMapEntry.java        |  205 ++--
 .../internal/processors/cache/GridCacheMvcc.java   |    5 +-
 .../processors/cache/GridCacheMvccEntryInfo.java   |    9 +
 .../processors/cache/GridCacheProcessor.java       |   55 +-
 .../processors/cache/GridCacheTtlManager.java      |   47 +-
 .../cache/IgniteCacheOffheapManager.java           |   55 +-
 .../cache/IgniteCacheOffheapManagerImpl.java       |  399 +++----
 .../processors/cache/IgniteRebalanceIterator.java  |    6 +
 .../cache/binary/BinaryMetadataTransport.java      |   28 +-
 .../binary/CacheObjectBinaryProcessorImpl.java     |   67 +-
 .../distributed/GridDistributedCacheEntry.java     |    2 +-
 .../cache/distributed/dht/GridDhtGetFuture.java    |   14 +
 .../distributed/dht/GridDhtGetSingleFuture.java    |   14 +
 .../dht/GridDhtTxAbstractEnlistFuture.java         |   29 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java    |    3 +-
 .../dht/GridDhtTxQueryFirstEnlistRequest.java      |    1 +
 .../dht/GridPartitionedSingleGetFuture.java        |   12 +-
 .../dht/preloader/GridDhtForceKeysFuture.java      |    2 +
 .../dht/preloader/GridDhtPartitionDemander.java    |  262 ++++-
 .../dht/preloader/GridDhtPartitionSupplier.java    |   85 +-
 .../preloader/GridDhtPartitionsExchangeFuture.java |  165 +--
 .../dht/preloader/GridDhtPreloader.java            |    4 +
 .../dht/preloader/IgniteRebalanceIteratorImpl.java |   31 +
 .../dht/preloader/latch/ExchangeLatchManager.java  |  146 ++-
 .../near/GridNearTxAbstractEnlistFuture.java       |   13 +
 .../processors/cache/mvcc/MvccProcessorImpl.java   |  188 ++-
 .../internal/processors/cache/mvcc/MvccUtils.java  |   11 +-
 .../processors/cache/mvcc/txlog/TxLog.java         |    7 +-
 .../cache/persistence/CacheDataRowAdapter.java     |   18 +-
 .../cache/persistence/DataRegionMetricsImpl.java   |   18 +-
 .../persistence/DataRegionMetricsMXBeanImpl.java   |    5 +
 .../persistence/DataRegionMetricsSnapshot.java     |    9 +
 .../GridCacheDatabaseSharedManager.java            |  138 ++-
 .../cache/persistence/GridCacheOffheapManager.java |   97 +-
 .../IgniteCacheDatabaseSharedManager.java          |   41 +-
 .../cache/persistence/file/FileDownloader.java     |  166 +--
 .../cache/persistence/file/FilePageStore.java      |    4 +-
 .../cache/persistence/file/FileUploader.java       |   40 +-
 .../cache/persistence/pagemem/PageMemoryImpl.java  |    2 +-
 .../cache/persistence/tree/io/DataPageIO.java      |   42 +-
 .../persistence/wal/FileWriteAheadLogManager.java  |   11 +-
 .../continuous/CacheContinuousQueryHandler.java    |   16 +-
 .../cache/transactions/IgniteTxHandler.java        |    6 +-
 .../IgniteTxImplicitSingleStateImpl.java           |   10 +-
 .../cache/transactions/IgniteTxManager.java        |    5 +-
 .../IgniteTxRemoteSingleStateImpl.java             |    5 -
 .../transactions/IgniteTxRemoteStateAdapter.java   |   14 +
 .../transactions/IgniteTxRemoteStateImpl.java      |   10 -
 .../processors/cache/tree/CacheDataRowStore.java   |    4 +-
 .../processors/cache/tree/CacheDataTree.java       |    2 +-
 .../cache/tree/mvcc/data/MvccDataRow.java          |   18 +-
 .../tree/mvcc/search/MvccSnapshotSearchRow.java    |    2 +-
 .../cluster/BaselineConfigurationMXBeanImpl.java   |   69 ++
 .../processors/cluster/ClusterProcessor.java       |   16 +-
 .../autoadjust/BaselineAutoAdjustData.java         |   45 +-
 .../autoadjust/BaselineAutoAdjustExecutor.java     |   14 +-
 .../autoadjust/BaselineAutoAdjustScheduler.java    |  134 ++-
 .../autoadjust/BaselineAutoAdjustStatistic.java    |   90 ++
 .../baseline/autoadjust/ChangeTopologyWatcher.java |   27 +-
 .../datastructures/AtomicDataStructureProxy.java   |   39 +-
 .../datastructures/GridCacheAtomicLongImpl.java    |   67 +-
 .../GridCacheAtomicReferenceImpl.java              |   36 +-
 .../datastructures/GridCacheAtomicStampedImpl.java |   37 +-
 .../odbc/jdbc/JdbcBatchExecuteRequest.java         |    2 +-
 .../odbc/jdbc/JdbcBatchExecuteResult.java          |    2 +-
 .../odbc/jdbc/JdbcConnectionContext.java           |   11 +-
 .../processors/odbc/jdbc/JdbcMetadataInfo.java     |   10 +-
 .../odbc/jdbc/JdbcOrderedBatchExecuteRequest.java  |    2 +-
 .../odbc/jdbc/JdbcOrderedBatchExecuteResult.java   |    2 +-
 .../odbc/jdbc/JdbcQueryExecuteRequest.java         |    2 +-
 .../odbc/jdbc/JdbcQueryExecuteResult.java          |    2 +-
 .../internal/processors/odbc/jdbc/JdbcRequest.java |    6 +
 .../processors/odbc/jdbc/JdbcRequestHandler.java   |   61 +-
 .../internal/processors/odbc/jdbc/JdbcResult.java  |    6 +
 .../odbc/odbc/OdbcConnectionContext.java           |    2 +
 .../processors/odbc/odbc/OdbcRequestHandler.java   |   22 +-
 .../platform/cluster/PlatformClusterGroup.java     |    1 +
 .../processors/query/GridQueryIndexing.java        |   23 +-
 .../processors/query/GridQueryProcessor.java       |   73 +-
 .../processors/query/QueryHistoryMetrics.java      |   15 +-
 .../processors/query/QueryHistoryMetricsValue.java |   10 +-
 .../processors/query/SqlClientContext.java         |   40 +-
 .../processors/service/IgniteServiceProcessor.java |    8 +-
 .../internal/util/GridReadOnlyArrayView.java       |   81 ++
 .../apache/ignite/internal/util/IgniteUtils.java   |  101 +-
 .../internal/util/future/GridFutureAdapter.java    |    4 +-
 .../baseline/VisorBaselineAutoAdjustSettings.java  |   30 +-
 .../internal/visor/baseline/VisorBaselineTask.java |   32 +-
 .../visor/baseline/VisorBaselineTaskResult.java    |   39 +-
 .../visor/baseline/VisorBaselineViewTask.java      |   11 +-
 .../internal/visor/query/VisorQueryPingTask.java   |   88 ++
 .../VisorQueryPingTaskResult.java}                 |   50 +-
 .../ignite/mxbean/BaselineConfigurationMXBean.java |   45 +
 .../ignite/mxbean/DataRegionMetricsMXBean.java     |    4 +
 .../ignite/spi/discovery/tcp/ClientImpl.java       |   62 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java       |  110 +-
 .../discovery/tcp/internal/TcpDiscoveryNode.java   |    1 +
 .../encryption/keystore/KeystoreEncryptionSpi.java |    4 +-
 modules/core/src/test/config/tests.properties      |    5 +-
 .../ignite/cache/NodeWithFilterRestartTest.java    |    6 +-
 .../ignite/cache/ResetLostPartitionTest.java       |    2 -
 .../affinity/AffinityDistributionLoggingTest.java  |   37 +-
 .../cache/affinity/AffinityHistoryCleanupTest.java |  138 ++-
 .../failure/FailureHandlingConfigurationTest.java  |   54 +-
 .../ConsistentIdImplicitlyExplicitlyTest.java      |   18 +-
 ...dCachePartitionExchangeManagerHistSizeTest.java |   23 +-
 .../ignite/internal/GridVersionSelfTest.java       |   11 +-
 ...ientReconnectAtomicsWithLostPartitionsTest.java |  372 ++++++
 .../ignite/internal/IgniteClientRejoinTest.java    |   14 +-
 .../IgniteDiscoveryMassiveNodeFailTest.java        |   19 +-
 ...iteUpdateNotifierPerClusterSettingSelfTest.java |   20 +-
 .../BinaryConfigurationConsistencySelfTest.java    |   28 +-
 .../BinarySimpleNameTestPropertySelfTest.java      |   19 +-
 .../commandline/CommandHandlerParsingTest.java     |    4 +-
 .../managers/IgniteDiagnosticMessagesTest.java     |    2 +
 .../GridHistoryAffinityAssignmentTest.java         |    6 -
 ...istoryAffinityAssignmentTestNoOptimization.java |    3 -
 .../cache/CacheGetRemoveSkipStoreTest.java         |  293 +++++
 .../processors/cache/CacheMetricsManageTest.java   |   32 +-
 .../processors/cache/CacheNamesSelfTest.java       |   85 --
 .../cache/CacheNamesWithSpecialCharactersTest.java |   74 --
 .../ClientFastReplyCoordinatorFailureTest.java     |  141 +++
 .../GridCacheConfigurationValidationSelfTest.java  |   64 +-
 .../GridCacheMixedPartitionExchangeSelfTest.java   |   10 +-
 .../GridCacheMvccMultiThreadedUpdateSelfTest.java  |   49 +-
 ...OffHeapMultiThreadedUpdateAbstractSelfTest.java |   23 +-
 .../processors/cache/GridCacheTestEntryEx.java     |   18 +-
 .../IgniteClientCacheInitializationFailTest.java   |   19 +-
 .../cache/IgniteClientCacheStartFailoverTest.java  |    5 +-
 .../IgniteMvccTxMultiThreadedAbstractTest.java     |    7 -
 .../IgniteMvccTxSingleThreadedAbstractTest.java    |    3 -
 .../cache/IgnitePdsDataRegionMetricsTxTest.java    |    4 -
 .../processors/cache/IgniteTxAbstractTest.java     |    8 +-
 .../cache/IgniteTxExceptionAbstractSelfTest.java   |   32 +-
 ...iteExchangeLatchManagerCoordinatorFailTest.java |   93 +-
 .../CacheLateAffinityAssignmentTest.java           |    4 -
 .../GridCachePreloadRestartAbstractSelfTest.java   |    2 +-
 .../IgniteCachePartitionLossPolicySelfTest.java    |   46 +
 ...WriteSynchronizationModesMultithreadedTest.java |   29 +-
 .../IgniteTxConcurrentRemoveObjectsTest.java       |   24 +-
 ...CacheColocatedMvccTxSingleThreadedSelfTest.java |    3 -
 .../dht/GridCacheColocatedTxExceptionSelfTest.java |   58 -
 .../dht/GridCacheDhtPreloadPutGetSelfTest.java     |   14 +-
 ...achePartitionedMvccTxMultiThreadedSelfTest.java |   12 +-
 ...chePartitionedMvccTxSingleThreadedSelfTest.java |    3 -
 ...achePartitionedOptimisticTxNodeRestartTest.java |    6 -
 .../near/NearCacheMultithreadedUpdateTest.java     |    2 +-
 .../GridCacheRebalancingSyncSelfTest.java          |   30 +-
 ...acheRebalancingUnmarshallingFailedSelfTest.java |   43 +-
 ...CacheReplicatedMvccTxMultiThreadedSelfTest.java |    4 +-
 .../GridCacheReplicatedTxExceptionSelfTest.java    |   58 -
 .../mvcc/CacheMvccConfigurationValidationTest.java |    6 +
 .../cache/mvcc/CacheMvccTransactionsTest.java      |  178 +--
 .../CommonPoolStarvationCheckpointTest.java        |  147 +++
 .../IgnitePdsBinarySortObjectFieldsTest.java       |  123 +-
 ...itePdsCacheStartStopWithFreqCheckpointTest.java |   13 +-
 .../IgnitePdsContinuousRestartTest.java            |    9 -
 .../persistence/IgnitePdsDynamicCacheTest.java     |   13 +-
 .../IgnitePdsPartitionsStateRecoveryTest.java      |    7 +-
 .../IgnitePdsRecoveryAfterFileCorruptionTest.java  |    6 +-
 .../IgnitePdsSporadicDataRecordsOnBackupTest.java  |  491 ++++----
 ...ocalWalModeChangeDuringRebalancingSelfTest.java |   11 +-
 .../ClientAffinityAssignmentWithBaselineTest.java  |   25 +-
 .../cache/persistence/db/IgnitePdsWithTtlTest.java |  118 +-
 .../db/checkpoint/CheckpointFreeListTest.java      |   10 +-
 ...CheckpointSimulationWithRealCpDisabledTest.java |    2 +-
 .../cache/persistence/file/FileDownloaderTest.java |   15 +-
 .../pagemem/UsedPagesMetricAbstractTest.java       |   97 ++
 .../persistence/pagemem/UsedPagesMetricTest.java   |   58 +
 .../pagemem/UsedPagesMetricTestPersistence.java    |   91 ++
 .../wal/ExplicitWalDeltaConsistencyTest.java       |    2 -
 .../PageMemoryTrackerPluginProvider.java           |    6 +-
 ...heContinuousQueryOperationFromCallbackTest.java |  292 ++++-
 .../CacheContinuousQueryOrderingEventTest.java     |   39 +-
 .../continuous/CacheKeepBinaryIterationTest.java   |    6 -
 .../IgniteCacheContinuousQueryBackupQueueTest.java |    7 +-
 .../IgniteCacheContinuousQueryClientTest.java      |    6 +
 .../cache/transactions/TxRollbackAsyncTest.java    |    4 +-
 .../cluster/BaselineConfigurationMXBeanTest.java   |   84 ++
 .../cluster/ChangeTopologyWatcherTest.java         |   25 +-
 .../database/IgniteDbDynamicCacheSelfTest.java     |   13 +-
 .../DistributedMetaStoragePersistentTest.java      |    4 +-
 ...ServiceHotRedeploymentViaDeploymentSpiTest.java |  161 +++
 .../spi/ExponentialBackoffTimeoutStrategyTest.java |    3 -
 .../TcpCommunicationSpiFaultyClientSslTest.java    |    3 -
 .../tcp/TcpCommunicationSpiFreezingClientTest.java |    3 -
 .../tcp/TcpDiscoveryCoordinatorFailureTest.java    |    3 -
 .../discovery/tcp/TcpDiscoveryFailedJoinTest.java  |    6 +-
 .../tcp/TcpDiscoveryNetworkIssuesTest.java         |  215 ++++
 .../apache/ignite/testframework/GridTestUtils.java |   98 +-
 .../ignite/testframework/MvccFeatureChecker.java   |    1 -
 .../testframework/config/GridTestProperties.java   |    3 -
 .../testframework/junits/GridAbstractTest.java     |  194 +--
 .../junits/GridAbstractTestWithAssumption.java     |   50 -
 .../testframework/junits/IgniteTestResources.java  |    4 +-
 .../junits/SystemPropertiesList.java}              |   21 +-
 .../testframework/junits/WithSystemProperty.java}  |   26 +-
 .../junits/multijvm/IgniteClusterProcessProxy.java |   25 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java    |    2 +
 ...bjectsSimpleNameMapperComputeGridTestSuite.java |    1 -
 ...IgniteBinarySimpleNameMapperBasicTestSuite.java |    1 -
 ...inarySimpleNameMapperCacheFullApiTestSuite.java |    1 -
 .../testsuites/IgniteCacheMvccTestSuite1.java      |    4 -
 .../testsuites/IgniteCacheMvccTestSuite4.java      |    2 +
 .../testsuites/IgniteCacheMvccTestSuite6.java      |    2 +
 .../testsuites/IgniteCacheRestartTestSuite.java    |    2 +
 .../ignite/testsuites/IgniteCacheTestSuite.java    |    4 -
 .../ignite/testsuites/IgniteCacheTestSuite4.java   |    2 +
 .../ignite/testsuites/IgniteCacheTestSuite6.java   |    2 +
 .../testsuites/IgniteClientReconnectTestSuite.java |    2 +
 .../ignite/testsuites/IgnitePdsTestSuite.java      |    4 +
 .../testsuites/IgniteServiceGridTestSuite.java     |    2 +
 .../IgniteSpiDiscoverySelfTestSuite.java           |    3 +
 .../apache/ignite/util/GridCommandHandlerTest.java |   72 +-
 modules/dev-utils/ignite-modules-test/README.md    |   16 +
 modules/dev-utils/ignite-modules-test/build.gradle |  139 +++
 .../src/test/java/module-info.java}                |   13 +-
 .../modulestest/IgniteLaunchInModularEnvTest.java  |   97 ++
 modules/extdata/uri/pom.xml                        |  118 ++
 .../uri/tasks/GridUriDeploymentDependency9.java    |   58 +
 .../uri/tasks/GridUriDeploymentTestTask10.java}    |   31 +-
 .../uri/tasks/GridUriDeploymentTestTask11.java}    |   31 +-
 .../uri/tasks/GridUriDeploymentTestTask8.java}     |   31 +-
 .../uri/tasks/GridUriDeploymentTestTask9.java      |   73 ++
 .../GridUriDeploymentTestWithNameTask10.java}      |   37 +-
 .../GridUriDeploymentTestWithNameTask11.java}      |   37 +-
 .../tasks/GridUriDeploymentTestWithNameTask8.java} |   37 +-
 .../ignite/spi/deployment/uri/tasks/spring9.xml    |   45 +
 .../spi/deployment/uri/tasks/test9.properties      |   22 +
 .../testsuites/IgniteBinaryHibernateTestSuite.java |   37 -
 .../IgniteBinaryHibernate5TestSuite.java           |   37 -
 .../IgniteBinaryHibernate53TestSuite.java          |   37 -
 .../processors/query/h2/CommandProcessor.java      |   26 +-
 .../processors/query/h2/IgniteH2Indexing.java      | 1229 ++++++++++----------
 ...eryParserCacheKey.java => QueryDescriptor.java} |  110 +-
 .../processors/query/h2/QueryParameters.java       |  215 ++++
 .../internal/processors/query/h2/QueryParser.java  |  149 ++-
 .../processors/query/h2/QueryParserResult.java     |   29 +-
 .../processors/query/h2/QueryParserResultDml.java  |   16 +-
 .../processors/query/h2/SchemaManager.java         |   14 +-
 .../internal/processors/query/h2/dml/DmlUtils.java |    5 +-
 .../processors/query/h2/dml/UpdatePlan.java        |   10 +-
 .../processors/query/h2/dml/UpdatePlanBuilder.java |   89 +-
 .../h2/sys/view/SqlAbstractLocalSystemView.java    |   56 +-
 .../h2/sys/view/SqlSystemViewBaselineNodes.java    |    2 +-
 .../h2/sys/view/SqlSystemViewCacheGroups.java      |   51 +-
 .../view/SqlSystemViewCacheGroupsIOStatistics.java |    8 +-
 .../query/h2/sys/view/SqlSystemViewCaches.java     |  147 +--
 .../query/h2/sys/view/SqlSystemViewIndexes.java    |   32 +-
 .../query/h2/sys/view/SqlSystemViewNodes.java      |    8 +-
 .../sys/view/SqlSystemViewQueryHistoryMetrics.java |   92 ++
 .../h2/sys/view/SqlSystemViewRunningQueries.java   |    4 +-
 .../query/h2/sys/view/SqlSystemViewTables.java     |   35 +-
 .../cache/BinaryTypeRegistrationTest.java          |  194 +++
 .../ClientReconnectAfterClusterRestartTest.java    |   39 +-
 .../cache/IgniteCacheInsertSqlQuerySelfTest.java   |   91 +-
 .../cache/IgniteCacheSqlDmlErrorSelfTest.java      |   21 +-
 .../IgniteCacheSqlInsertValidationSelfTest.java    |   19 +-
 .../cache/IgniteCacheUpdateSqlQuerySelfTest.java   |  387 ++++--
 .../cache/StartCachesInParallelTest.java           |   43 +-
 ...tributedPartitionQueryNodeRestartsSelfTest.java |   22 +-
 ...cheQueryNodeRestartDistributedJoinSelfTest.java |   27 +-
 .../index/DynamicIndexAbstractBasicSelfTest.java   |   67 +-
 .../cache/index/H2DynamicTableSelfTest.java        |   36 +-
 ...ransactionsCommandsWithMvccEnabledSelfTest.java |    2 +-
 ...StaticCacheDdlKeepStaticConfigurationTest.java} |   16 +-
 .../processors/cache/index/StaticCacheDdlTest.java |  402 +++++++
 .../mvcc/CacheMvccBasicContinuousQueryTest.java    |   36 +-
 .../mvcc/CacheMvccContinuousQueryClientTest.java   |    9 -
 .../mvcc/CacheMvccPartitionedSqlQueriesTest.java   |    2 +-
 .../mvcc/CacheMvccReplicatedSqlQueriesTest.java    |    2 +-
 .../mvcc/CacheMvccSqlQueriesAbstractTest.java      |   99 +-
 .../mvcc/CacheMvccSqlTxQueriesAbstractTest.java    |   68 +-
 .../mvcc/MvccDeadlockDetectionConfigTest.java      |    3 -
 .../cache/mvcc/MvccDeadlockDetectionTest.java      |    4 +-
 .../persistence/db/wal/IgniteWalRecoveryTest.java  |  140 ++-
 .../transaction/DmlInsideTransactionTest.java      |   26 +-
 ...tStoreQueryWithMultipleClassesPerCacheTest.java |    8 +-
 .../IgnitePersistentStoreSchemaLoadTest.java       |    6 +-
 .../query/IgniteSqlKeyValueFieldsTest.java         |   14 +-
 .../processors/query/LocalQueryLazyTest.java       |  108 ++
 .../processors/query/RunningQueriesTest.java       |  105 +-
 .../SqlIncompatibleDataTypeExceptionTest.java      |  199 ++++
 .../processors/query/SqlQueryHistorySelfTest.java  |    2 +-
 .../processors/query/SqlSystemViewsSelfTest.java   |  377 ++++--
 .../processors/query/h2/QueryDataPageScanTest.java |    5 +-
 .../twostep/AbstractPartitionPruningBaseTest.java  |    3 -
 .../DisappearedCacheCauseRetryMessageSelfTest.java |    4 +-
 ...DisappearedCacheWasNotFoundMessageSelfTest.java |    4 +-
 .../twostep/DmlSelectPartitionPruningSelfTest.java |    6 +-
 .../twostep/MvccDmlPartitionPruningSelfTest.java   |    3 -
 .../twostep/NonCollocatedRetryMessageSelfTest.java |   17 +-
 .../h2/twostep/RetryCauseMessageSelfTest.java      |    4 +-
 .../processors/query/oom/AbstractQueryOOMTest.java |   72 +-
 .../oom/QueryOOMWithQueryParallelismTest.java      |    4 -
 .../oom/QueryOOMWithoutQueryParallelismTest.java   |    4 -
 .../IgniteBinaryCacheQueryTestSuite.java           |    3 +
 .../IgniteBinaryCacheQueryTestSuite2.java          |    3 +
 .../testsuites/IgniteCacheQuerySelfTestSuite6.java |    6 +-
 .../IgniteCacheWithIndexingTestSuite.java          |    3 +
 .../vector/impl/DelegatingNamedVector.java         |   24 +-
 modules/platforms/cpp/core/src/ignition.cpp        |   13 +-
 modules/platforms/cpp/jni/Makefile.am              |    1 +
 .../platforms/cpp/jni/include/ignite/jni/java.h    |    7 +
 modules/platforms/cpp/jni/os/linux/src/utils.cpp   |   10 +-
 modules/platforms/cpp/jni/src/java.cpp             |   79 +-
 modules/platforms/cpp/odbc/src/message.cpp         |    3 +-
 .../cpp/odbc/src/streaming/streaming_context.cpp   |    2 +
 modules/platforms/cpp/project/vs/ignite.slnrel     |   10 +
 modules/platforms/cpp/project/vs/ignite_x86.slnrel |   10 +
 .../ApiParity/ClusterParityTest.cs                 |    4 +-
 .../Apache.Ignite.Core/IDataRegionMetrics.cs       |   21 +-
 .../Apache.Ignite.Core/Impl/DataRegionMetrics.cs   |   20 +-
 .../python/pyignite/datatypes/standard.py          |   13 +-
 modules/platforms/python/tests/test_datatypes.py   |   42 +
 modules/rest-http/pom.xml                          |   12 +-
 .../GridSpringBeanSerializationSelfTest.java       |    4 +
 modules/sqlline/bin/sqlline.bat                    |    2 +-
 modules/sqlline/bin/sqlline.sh                     |    2 +-
 .../uri/GridUriDeploymentClassLoader.java          |   47 +-
 .../uri/GridUriDeploymentClassLoaderFactory.java   |    8 +-
 .../uri/GridUriDeploymentFileProcessor.java        |   59 +-
 .../uri/GridUriDeploymentFileProcessorResult.java  |    8 +-
 .../uri/GridUriDeploymentJarVerifier.java          |    4 +-
 .../uri/GridUriDeploymentSpringDocument.java       |    2 +-
 .../uri/GridUriDeploymentUnitDescriptor.java       |   15 +-
 .../spi/deployment/uri/UriDeploymentSpi.java       |   72 +-
 .../scanners/file/UriDeploymentFileScanner.java    |   54 +-
 ...DeploymentClassLoaderMultiThreadedSelfTest.java |    2 +-
 .../scanners/file/GridFileDeploymentSelfTest.java  |   52 +-
 .../scanners/http/GridHttpDeploymentSelfTest.java  |   43 +-
 modules/web-console/DEVNOTES.txt                   |    2 +-
 modules/web-console/backend/app/browsersHandler.js |    1 +
 modules/web-console/backend/app/schemas.js         |    5 +-
 modules/web-console/backend/index.js               |    0
 modules/web-console/backend/package.json           |    3 +-
 modules/web-console/backend/routes/admin.js        |    5 +-
 modules/web-console/backend/services/auth.js       |    6 +-
 modules/web-console/backend/services/mails.js      |  158 ++-
 modules/web-console/backend/services/users.js      |   22 +-
 modules/web-console/backend/templates/base.html    |   21 +
 modules/web-console/frontend/app/app.js            |    1 -
 .../frontend/app/components/breadcrumbs/style.scss |    5 +-
 .../app/components/cluster-selector/controller.js  |   10 +-
 .../app/components/cluster-selector/template.pug   |    4 +-
 .../component.spec.js                              |    8 +-
 .../frontend/app/components/page-landing/index.js  |   13 +-
 .../app/components/page-landing/template.pug       |    4 +-
 .../components/queries-notebook/controller.ts      |  668 +++++++----
 .../components/queries-notebook/template.tpl.pug   |    8 +-
 .../components/user-menu/style.scss                |   15 +
 .../components/user-menu/template.pug              |    4 +-
 .../components/modal-import-models/component.js    |    5 +-
 .../page-configure-basic/controller.spec.js        |  206 ----
 .../page-configure-basic/reducer.spec.js           |   19 +-
 .../components/page-configure-basic/reducer.ts     |   22 +-
 .../generator/generator/JavaTransformer.spec.js    |   55 -
 .../generator/generator/SharpTransformer.spec.js   |   53 -
 .../generator/generator/SpringTransformer.spec.js  |   54 -
 .../services/ConfigurationDownload.spec.js         |   22 +-
 .../configuration/services/PageConfigure.spec.js   |  240 ----
 .../app/configuration/store/reducer.spec.js        |   31 +-
 .../app/modules/agent/AgentManager.service.js      |  125 +-
 .../frontend/app/services/Version.service.js       |  104 +-
 .../frontend/app/services/Version.spec.js          |   18 +-
 modules/web-console/frontend/app/vendor.js         |    3 -
 modules/web-console/frontend/package.json          |    5 -
 .../public/images/page-landing-carousel-2.png      |  Bin 34564 -> 0 bytes
 .../public/images/page-landing-carousel-3.png      |  Bin 27808 -> 0 bytes
 ...g-carousel-1.png => page-landing-ui-sample.png} |  Bin
 .../frontend/public/stylesheets/style.scss         |   62 +-
 modules/web-console/frontend/views/index.pug       |    2 +
 modules/web-console/pom.xml                        |   12 +-
 .../web-console/web-agent/bin/ignite-web-agent.bat |    2 +-
 .../web-console/web-agent/bin/ignite-web-agent.sh  |    2 +-
 .../org/apache/ignite/yarn/ClusterProperties.java  |   65 +-
 .../org/apache/ignite/yarn/IgniteYarnClient.java   |    4 +-
 .../yarn/IgniteApplicationMasterSelfTest.java      |   14 +
 .../zk/internal/ZookeeperDiscoveryImpl.java        |    6 +-
 .../tcp/ipfinder/zk/ZookeeperIpFinderTest.java     |    5 +-
 .../zk/ZookeeperDiscoverySpiTestSuite1.java        |   14 +-
 .../ZookeeperDiscoveryClientReconnectTest.java     |   15 +-
 ...erDiscoveryConcurrentStartAndStartStopTest.java |   13 +-
 ...coverySegmentationAndConnectionRestoreTest.java |  108 +-
 .../ZookeeperDiscoverySpiSaslFailedAuthTest.java   |   13 +-
 ...ookeeperDiscoverySpiSaslSuccessfulAuthTest.java |    8 +-
 .../zk/internal/ZookeeperDiscoverySpiTestBase.java |    4 +-
 parent/pom.xml                                     |   10 +-
 pom.xml                                            |    1 -
 444 files changed, 13316 insertions(+), 7308 deletions(-)

diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index a69157e,04443be..d955cf7
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
@@@ -39,10 -36,8 +39,9 @@@ import javax.cache.processor.EntryProce
  import org.apache.ignite.IgniteCheckedException;
  import org.apache.ignite.IgniteException;
  import org.apache.ignite.IgniteLogger;
--import org.apache.ignite.IgniteSystemProperties;
  import org.apache.ignite.internal.NodeStoppingException;
 +import org.apache.ignite.internal.UnregisteredBinaryTypeException;
 +import org.apache.ignite.internal.UnregisteredClassException;
  import org.apache.ignite.internal.pagemem.FullPageId;
  import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageMvccMarkUpdatedRecord;
  import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageMvccUpdateNewTxStateHintRecord;
@@@ -1673,57 -1594,19 +1636,65 @@@ public class IgniteCacheOffheapManagerI
          /** {@inheritDoc} */
          @Override public void invoke(GridCacheContext cctx, KeyCacheObject key, OffheapInvokeClosure c)
              throws IgniteCheckedException {
+             if (!busyLock.enterBusy())
+                 throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+ 
              int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID;
  
-             invoke0(cctx, new SearchRow(cacheId, key), c);
+             try {
+                 invoke0(cctx, new SearchRow(cacheId, key), c);
+             }
+             finally {
+                 busyLock.leaveBusy();
+             }
          }
  
 +        /** {@inheritDoc} */
 +        @Override public SearchRowEx createSearchRow(GridCacheContext cctx, KeyCacheObject key, Object data) {
 +            int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID;
 +
 +            return new SearchRowEx<>(cacheId, key, data);
 +        }
 +
 +        /** {@inheritDoc} */
 +        @Override public Comparator<CacheSearchRow> rowsComparator() {
 +            return dataTree.rowsComparator();
 +        }
 +
 +        /** {@inheritDoc} */
 +        @Override public void invokeAll(GridCacheContext cctx,
 +                Collection<? extends CacheSearchRow> rows,
 +                Function<CacheSearchRow, OffheapInvokeClosure> closures)
 +                throws IgniteCheckedException {
 +            if (!busyLock.enterBusy())
 +                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
 +
 +            try {
 +                RuntimeException err = null;
 +
 +                try {
 +                    dataTree.invokeAll(rows.iterator(), CacheDataRowAdapter.RowData.NO_KEY, closures);
 +                }
 +                catch (UnregisteredClassException | UnregisteredBinaryTypeException clsErr) {
 +                    err = clsErr;
 +                }
 +
 +                for (CacheSearchRow row : rows) {
 +                    OffheapInvokeClosure c = closures.apply(row);
 +
 +                    // Update could be interrupted in the middle, finish update only for processed entries.
 +                    if (c.operationType() != null)
-                         finishInvoke(cctx, row.key(), c);
++                        finishInvoke(cctx, row, c);
 +                }
 +
 +                if (err != null)
 +                    throw err;
 +            }
 +            finally {
 +                busyLock.leaveBusy();
 +            }
 +        }
 +
          /**
           * @param cctx Cache context.
           * @param row Search row.
@@@ -1732,29 -1615,10 +1703,14 @@@
           */
          private void invoke0(GridCacheContext cctx, CacheSearchRow row, OffheapInvokeClosure c)
              throws IgniteCheckedException {
-             if (!busyLock.enterBusy())
-                 throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
- 
-             try {
-                 assert cctx.shared().database().checkpointLockIsHeldByThread();
+             assert cctx.shared().database().checkpointLockIsHeldByThread();
  
-                 dataTree.invoke(row, CacheDataRowAdapter.RowData.NO_KEY, c);
+             dataTree.invoke(row, CacheDataRowAdapter.RowData.NO_KEY, c);
  
-                 finishInvoke(cctx, row.key(), c);
-             }
-             finally {
-                 busyLock.leaveBusy();
-             }
 +        }
 +
-         /**
-          * @param cctx Cache context.
-          * @param key Key.
-          * @param c Closure.
-          * @throws IgniteCheckedException If failed.
-          */
-         private void finishInvoke(GridCacheContext cctx, KeyCacheObject key, OffheapInvokeClosure c)
++        private void finishInvoke(GridCacheContext cctx, CacheSearchRow row, OffheapInvokeClosure c)
 +            throws IgniteCheckedException {
              switch (c.operationType()) {
                  case PUT: {
                      assert c.newRow() != null : c;