Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 939CA200C86 for ; Wed, 31 May 2017 11:22:56 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 8F56A160BBA; Wed, 31 May 2017 09:22:56 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 7D450160BE9 for ; Wed, 31 May 2017 11:22:53 +0200 (CEST) Received: (qmail 78415 invoked by uid 500); 31 May 2017 09:22:52 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 78213 invoked by uid 99); 31 May 2017 09:22:52 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 31 May 2017 09:22:52 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 49656E03B3; Wed, 31 May 2017 09:22:52 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sboikov@apache.org To: commits@ignite.apache.org Date: Wed, 31 May 2017 09:23:40 -0000 Message-Id: <613c22da680742ac814c63347b78c2e4@git.apache.org> In-Reply-To: <35e76c131ae44a50b1bcfcbec4281e1d@git.apache.org> References: <35e76c131ae44a50b1bcfcbec4281e1d@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [50/51] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-5075' into ignite-5075-pds archived-at: Wed, 31 May 2017 09:22:56 -0000 Merge remote-tracking branch 'remotes/origin/ignite-5075' into ignite-5075-pds # Conflicts: # modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java # modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOffheapEvictionManager.java # modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java # modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java # modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java # modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java # modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java # modules/web-console/frontend/app/modules/agent/AgentManager.service.js Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/1305ce9c Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/1305ce9c Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/1305ce9c Branch: refs/heads/ignite-5075-pds Commit: 1305ce9c6e5a0f6e2855faa7be7f3212afbec161 Parents: 815a92f 6250119 Author: sboikov Authored: Wed May 31 12:21:09 2017 +0300 Committer: sboikov Committed: Wed May 31 12:21:09 2017 +0300 ---------------------------------------------------------------------- .../datagrid/CacheClientBinaryQueryExample.java | 4 +- .../jdbc2/JdbcAbstractDmlStatementSelfTest.java | 50 +- .../JdbcAbstractUpdateStatementSelfTest.java | 11 +- .../ignite/internal/jdbc2/JdbcBlobTest.java | 485 +++++++++++ .../jdbc2/JdbcInsertStatementSelfTest.java | 16 +- .../jdbc2/JdbcMergeStatementSelfTest.java | 16 +- .../internal/jdbc2/JdbcNoDefaultCacheTest.java | 2 - .../jdbc2/JdbcPreparedStatementSelfTest.java | 47 ++ .../jdbc/suite/IgniteJdbcDriverTestSuite.java | 7 +- .../cache/affinity/AffinityKeyMapper.java | 3 + .../configuration/CacheConfiguration.java | 11 + .../ignite/internal/GridKernalContext.java | 1 + .../ignite/internal/GridKernalContextImpl.java | 1 + .../org/apache/ignite/internal/IgniteEx.java | 9 +- .../apache/ignite/internal/IgniteKernal.java | 19 +- .../internal/binary/BinaryEnumObjectImpl.java | 11 +- .../internal/binary/BinaryObjectImpl.java | 24 +- .../binary/BinaryObjectOffheapImpl.java | 9 +- .../apache/ignite/internal/jdbc2/JdbcBlob.java | 191 +++++ .../ignite/internal/jdbc2/JdbcConnection.java | 2 +- .../internal/jdbc2/JdbcPreparedStatement.java | 4 +- .../ignite/internal/jdbc2/JdbcResultSet.java | 8 +- .../internal/processors/cache/CacheData.java | 13 + .../cache/CacheGroupInfrastructure.java | 13 +- .../cache/CacheJoinNodeDiscoveryData.java | 15 +- .../internal/processors/cache/CacheObject.java | 8 +- .../processors/cache/CacheObjectAdapter.java | 4 +- .../cache/CacheObjectByteArrayImpl.java | 8 +- .../processors/cache/CacheObjectContext.java | 197 +---- .../processors/cache/CacheObjectImpl.java | 31 +- .../processors/cache/CacheObjectUtils.java | 173 ++++ .../cache/CacheObjectValueContext.java | 50 ++ .../cache/CacheOffheapEvictionManager.java | 11 +- .../processors/cache/ClusterCachesInfo.java | 64 +- .../cache/DynamicCacheChangeRequest.java | 21 +- .../cache/DynamicCacheDescriptor.java | 13 + .../processors/cache/GridCacheAdapter.java | 4 +- .../processors/cache/GridCacheAttributes.java | 21 +- .../cache/GridCacheConcurrentMap.java | 32 +- .../cache/GridCacheConcurrentMapImpl.java | 56 +- .../processors/cache/GridCacheContext.java | 24 +- .../processors/cache/GridCacheEventManager.java | 2 +- .../processors/cache/GridCacheIoManager.java | 6 +- .../cache/GridCacheLocalConcurrentMap.java | 23 +- .../processors/cache/GridCacheMapEntry.java | 27 +- .../GridCachePartitionExchangeManager.java | 2 +- .../processors/cache/GridCacheProcessor.java | 85 +- .../processors/cache/GridNoStorageCacheMap.java | 4 +- .../cache/IgniteCacheOffheapManagerImpl.java | 15 +- .../processors/cache/IgniteCacheProxy.java | 2 +- .../processors/cache/KeyCacheObjectImpl.java | 10 +- .../cache/binary/CacheObjectBinaryContext.java | 6 +- .../binary/CacheObjectBinaryProcessorImpl.java | 28 +- .../cache/database/CacheDataRowAdapter.java | 8 +- .../dht/GridCachePartitionedConcurrentMap.java | 8 +- .../dht/GridClientPartitionTopology.java | 31 +- .../distributed/dht/GridDhtCacheEntry.java | 10 +- .../distributed/dht/GridDhtLocalPartition.java | 237 +++--- .../dht/GridDhtPartitionTopology.java | 9 +- .../dht/GridDhtPartitionTopologyImpl.java | 59 +- .../GridDhtPartitionsExchangeFuture.java | 46 +- .../cache/distributed/near/GridNearTxLocal.java | 3 +- .../cache/query/GridCacheQueryRequest.java | 14 +- .../CacheContinuousQueryAcknowledgeBuffer.java | 120 +++ .../CacheContinuousQueryDeployableObject.java | 110 +++ .../continuous/CacheContinuousQueryEntry.java | 117 ++- .../CacheContinuousQueryEventBuffer.java | 486 +++++++++++ .../continuous/CacheContinuousQueryHandler.java | 833 +++++-------------- .../CacheContinuousQueryHandlerV2.java | 6 +- .../CacheContinuousQueryListener.java | 17 + .../continuous/CacheContinuousQueryManager.java | 32 +- .../CacheContinuousQueryPartitionRecovery.java | 267 ++++++ .../query/continuous/CounterSkipContext.java | 56 +- .../cache/transactions/TxDeadlock.java | 7 +- .../cacheobject/IgniteCacheObjectProcessor.java | 6 +- .../IgniteCacheObjectProcessorImpl.java | 39 +- .../continuous/GridContinuousBatchAdapter.java | 2 +- .../continuous/GridContinuousProcessor.java | 19 +- .../continuous/GridContinuousQueryBatch.java | 16 +- .../datastructures/DataStructuresProcessor.java | 1 + .../datastructures/GridCacheLockImpl.java | 10 +- .../query/CacheQueryObjectValueContext.java | 64 ++ .../query/GridQueryCacheObjectsIterator.java | 16 +- .../processors/query/GridQueryIndexing.java | 18 +- .../processors/query/GridQueryProcessor.java | 82 +- .../query/GridQueryTypeDescriptor.java | 15 + .../processors/query/GridRunningQueryInfo.java | 16 +- .../processors/query/QueryIndexKey.java | 28 +- .../internal/processors/query/QuerySchema.java | 2 +- .../query/QueryTypeDescriptorImpl.java | 9 + .../internal/processors/query/QueryUtils.java | 278 +++++-- .../query/property/QueryBinaryProperty.java | 2 +- .../handlers/cache/GridCacheCommandHandler.java | 2 +- .../query/VisorRunningQueriesCollectorTask.java | 2 +- .../ignite/spi/discovery/DiscoverySpi.java | 9 +- .../spi/discovery/tcp/TcpDiscoverySpi.java | 8 +- .../internal/GridAffinityNoCacheSelfTest.java | 10 +- .../processors/cache/IgniteCacheGroupsTest.java | 619 +++++++++++++- .../cache/IgniteDynamicCacheStartSelfTest.java | 4 +- .../IgniteIncompleteCacheObjectSelfTest.java | 10 +- .../BinaryAtomicCacheLocalEntriesSelfTest.java | 30 + .../BinaryTxCacheLocalEntriesSelfTest.java | 84 ++ .../BinaryAtomicCacheLocalEntriesSelfTest.java | 30 - .../BinaryTxCacheLocalEntriesSelfTest.java | 84 -- ...tinuousQueryAsyncFailoverAtomicSelfTest.java | 1 - ...nuousQueryConcurrentPartitionUpdateTest.java | 373 +++++++++ .../CacheContinuousQueryEventBufferTest.java | 217 +++++ ...ContinuousQueryFailoverAbstractSelfTest.java | 79 +- ...niteCacheContinuousQueryBackupQueueTest.java | 13 +- ...eCacheContinuousQueryImmutableEntryTest.java | 6 +- .../database/FreeListImplSelfTest.java | 10 +- .../processors/igfs/IgfsIgniteMock.java | 4 +- .../spi/discovery/tcp/TcpDiscoverySelfTest.java | 4 +- .../junits/multijvm/IgniteProcessProxy.java | 4 +- .../IgniteBinaryObjectsTestSuite.java | 4 +- .../query/h2/H2IndexingAbstractGeoSelfTest.java | 26 +- .../query/h2/H2IndexingBinaryGeoSelfTest.java | 30 - .../H2IndexingBinarySegmentedGeoSelfTest.java | 30 - .../query/h2/H2IndexingGeoSelfTest.java | 2 +- .../h2/H2IndexingSegmentedGeoSelfTest.java | 2 +- .../testsuites/GeoSpatialIndexingTestSuite.java | 5 - .../query/h2/DmlStatementsProcessor.java | 32 +- .../processors/query/h2/H2DmlPlanKey.java | 66 ++ .../query/h2/H2ResultSetIterator.java | 4 +- .../processors/query/h2/H2RowDescriptor.java | 27 +- .../internal/processors/query/h2/H2Schema.java | 46 +- .../processors/query/h2/H2TableDescriptor.java | 52 +- .../processors/query/h2/H2TableEngine.java | 2 +- .../query/h2/H2TwoStepCachedQuery.java | 18 +- .../query/h2/H2TwoStepCachedQueryKey.java | 19 +- .../internal/processors/query/h2/H2Utils.java | 79 +- .../processors/query/h2/IgniteH2Indexing.java | 199 ++--- .../query/h2/ddl/DdlStatementsProcessor.java | 66 +- .../processors/query/h2/opt/GridH2Table.java | 6 +- .../query/h2/opt/GridH2ValueCacheObject.java | 32 +- .../query/h2/opt/GridLuceneIndex.java | 2 +- .../query/h2/twostep/GridMapQueryExecutor.java | 20 +- .../h2/twostep/GridReduceQueryExecutor.java | 167 ++-- .../query/h2/twostep/ReduceQueryRun.java | 157 ++++ .../query/h2/twostep/msg/GridH2CacheObject.java | 39 +- .../IgniteCacheInsertSqlQuerySelfTest.java | 27 +- .../cache/index/AbstractSchemaSelfTest.java | 123 ++- .../DynamicIndexAbstractBasicSelfTest.java | 77 +- .../DynamicIndexAbstractConcurrentSelfTest.java | 36 +- .../index/DynamicIndexAbstractSelfTest.java | 30 +- .../index/H2DynamicIndexAbstractSelfTest.java | 48 +- .../cache/index/H2DynamicTableSelfTest.java | 79 +- .../cache/index/SchemaExchangeSelfTest.java | 13 +- .../query/IgniteSqlSchemaIndexingTest.java | 18 +- .../query/IgniteSqlSplitterSelfTest.java | 59 +- .../h2/GridIndexingSpiAbstractSelfTest.java | 19 +- .../query/h2/sql/BaseH2CompareQueryTest.java | 12 +- .../query/h2/sql/GridQueryParsingTest.java | 4 +- .../query/h2/sql/H2CompareBigQueryTest.java | 2 +- .../IgniteCacheQuerySelfTestSuite3.java | 5 + .../processors/query/h2/sql/bigQuery.sql | 8 +- modules/platforms/cpp/binary/Makefile.am | 67 +- .../platforms/cpp/binary/include/Makefile.am | 55 +- .../binary/include/ignite/binary/binary_type.h | 2 +- .../ignite/impl/binary/binary_type_impl.h | 61 ++ .../ignite/impl/binary/binary_writer_impl.h | 4 +- .../cpp/binary/project/vs/binary.vcxproj | 2 + .../binary/project/vs/binary.vcxproj.filters | 6 + .../binary/src/impl/binary/binary_type_impl.cpp | 61 ++ modules/platforms/cpp/common/Makefile.am | 54 +- .../platforms/cpp/common/include/Makefile.am | 1 + .../common/include/ignite/common/cancelable.h | 65 ++ .../cpp/common/include/ignite/common/promise.h | 18 + .../common/include/ignite/common/shared_state.h | 51 ++ .../cpp/common/include/ignite/future.h | 60 +- .../cpp/common/include/ignite/ignite_error.h | 3 + .../cpp/common/project/vs/common.vcxproj | 1 + .../common/project/vs/common.vcxproj.filters | 3 + modules/platforms/cpp/core-test/Makefile.am | 101 +-- .../cpp/core-test/include/ignite/test_utils.h | 24 +- .../cpp/core-test/project/vs/core-test.vcxproj | 1 + .../project/vs/core-test.vcxproj.filters | 10 +- .../cpp/core-test/src/cache_invoke_test.cpp | 2 +- .../cpp/core-test/src/cluster_test.cpp | 2 +- .../cpp/core-test/src/compute_test.cpp | 337 ++++++++ .../cpp/core-test/src/continuous_query_test.cpp | 2 +- .../platforms/cpp/core-test/src/test_utils.cpp | 9 + modules/platforms/cpp/core/Makefile.am | 82 +- modules/platforms/cpp/core/include/Makefile.am | 107 +-- .../cpp/core/include/ignite/cache/cache.h | 4 +- .../cpp/core/include/ignite/cache/query/query.h | 1 - .../cpp/core/include/ignite/compute/compute.h | 136 +++ .../core/include/ignite/compute/compute_func.h | 65 ++ .../platforms/cpp/core/include/ignite/ignite.h | 19 +- .../cpp/core/include/ignite/ignite_binding.h | 25 + .../include/ignite/ignite_binding_context.h | 2 +- .../cpp/core/include/ignite/impl/bindings.h | 24 + .../ignite/impl/cluster/cluster_group_impl.h | 31 +- .../ignite/impl/compute/cancelable_impl.h | 70 ++ .../include/ignite/impl/compute/compute_impl.h | 118 +++ .../ignite/impl/compute/compute_job_holder.h | 139 ++++ .../ignite/impl/compute/compute_job_result.h | 161 ++++ .../ignite/impl/compute/compute_task_holder.h | 213 +++++ .../include/ignite/impl/ignite_binding_impl.h | 2 + .../include/ignite/impl/ignite_environment.h | 69 ++ .../cpp/core/include/ignite/impl/ignite_impl.h | 23 +- .../ignite/impl/interop/interop_target.h | 11 +- .../platforms/cpp/core/project/vs/core.vcxproj | 9 + .../cpp/core/project/vs/core.vcxproj.filters | 33 + modules/platforms/cpp/core/src/ignite.cpp | 5 + .../src/impl/cluster/cluster_group_impl.cpp | 23 +- .../core/src/impl/compute/cancelable_impl.cpp | 59 ++ .../cpp/core/src/impl/compute/compute_impl.cpp | 35 + .../cpp/core/src/impl/ignite_environment.cpp | 260 +++++- .../platforms/cpp/core/src/impl/ignite_impl.cpp | 15 +- .../core/src/impl/interop/interop_target.cpp | 20 + .../src/impl/transactions/transactions_impl.cpp | 10 - modules/platforms/cpp/examples/Makefile.am | 2 +- modules/platforms/cpp/examples/configure.ac | 2 +- .../cpp/examples/odbc-example/Makefile.am | 12 +- .../cpp/examples/project/vs/ignite-examples.sln | 12 +- .../cpp/examples/put-get-example/Makefile.am | 58 ++ .../put-get-example/config/example-cache.xml | 67 ++ .../project/vs/put-get-example.vcxproj | 110 +++ .../project/vs/put-get-example.vcxproj.filters | 38 + .../put-get-example/src/put_get_example.cpp | 126 +++ .../cpp/examples/putget-example/Makefile.am | 58 -- .../putget-example/config/example-cache.xml | 67 -- .../project/vs/putget-example.vcxproj | 110 --- .../project/vs/putget-example.vcxproj.filters | 38 - .../putget-example/src/putget_example.cpp | 126 --- .../cpp/examples/query-example/Makefile.am | 12 +- .../platforms/cpp/jni/include/ignite/jni/java.h | 1 + modules/platforms/cpp/jni/src/java.cpp | 10 + .../StartupTest.cs | 78 ++ .../Apache.Ignite.Core.Tests.NuGet/TestUtil.cs | 30 + .../Apache.Ignite.Core.Tests.csproj | 10 + .../Binary/TypeNameParserTest.cs | 8 +- .../Cache/CacheConfigurationTest.cs | 6 +- .../Cache/Query/CacheLinqTest.cs | 131 ++- .../Cache/Query/CacheLinqTestSqlEscapeAll.cs | 34 + .../Compute/BinarizableClosureTaskTest.cs | 7 +- .../Compute/ClosureTaskTest.cs | 6 +- .../Compute/ComputeApiTest.cs | 20 +- .../Compute/IgniteExceptionTaskSelfTest.cs | 8 +- .../Compute/SerializableClosureTaskTest.cs | 5 +- .../Deployment/GetAddressFunc.cs | 35 + .../PeerAssemblyLoadingAllApisTest.cs | 167 ++++ .../Deployment/PeerAssemblyLoadingTest.cs | 192 +++++ .../PeerAssemblyLoadingVersioningTest.cs | 164 ++++ .../Deployment/ProcessNameFunc.cs | 50 ++ .../Deployment/ProcessNameTask.cs | 74 ++ .../Deployment/peer_assembly_app.config | 35 + .../IgniteConfigurationSerializerTest.cs | 10 +- .../Log/CustomLoggerTest.cs | 5 +- .../Process/IgniteProcess.cs | 12 +- .../Apache.Ignite.Core.csproj | 10 + .../Apache.Ignite.Core.nuspec | 7 +- .../Cache/Configuration/QueryEntity.cs | 64 +- .../Deployment/PeerAssemblyLoadingMode.cs | 55 ++ .../Apache.Ignite.Core/IgniteConfiguration.cs | 16 + .../IgniteConfigurationSection.xsd | 12 + .../dotnet/Apache.Ignite.Core/Ignition.cs | 18 +- .../Impl/Binary/BinarizableSerializer.cs | 5 +- .../Apache.Ignite.Core/Impl/Binary/Binary.cs | 1 + .../Binary/BinaryEqualityComparerSerializer.cs | 1 - .../Impl/Binary/BinaryProcessor.cs | 2 +- .../Impl/Binary/BinaryReader.cs | 66 +- .../Impl/Binary/BinaryReflectiveActions.cs | 7 +- .../BinaryReflectiveSerializerInternal.cs | 5 +- .../Impl/Binary/BinarySystemTypeSerializer.cs | 2 +- .../Impl/Binary/BinaryWriter.cs | 24 + .../Impl/Binary/BinaryWriterExtensions.cs | 9 +- .../Impl/Binary/IBinarySerializerInternal.cs | 10 +- .../Impl/Binary/Marshaller.cs | 37 +- .../Impl/Binary/SerializableSerializer.cs | 14 +- .../Impl/Binary/TypeResolver.cs | 6 +- .../Impl/Binary/UserSerializerProxy.cs | 5 +- .../Impl/Cache/IQueryEntityInternal.cs | 31 + .../Common/CopyOnWriteConcurrentDictionary.cs | 8 - .../Apache.Ignite.Core/Impl/Common/Future.cs | 13 +- .../Impl/Common/IgniteHome.cs | 3 +- .../Impl/Common/LoadedAssembliesResolver.cs | 8 +- .../Impl/Compute/Closure/ComputeActionJob.cs | 7 +- .../Impl/Compute/Closure/ComputeFuncJob.cs | 5 +- .../Impl/Compute/ComputeFunc.cs | 5 +- .../Impl/Compute/ComputeImpl.cs | 1 - .../Impl/Compute/ComputeJob.cs | 3 +- .../Impl/Compute/ComputeOutFunc.cs | 3 +- .../Impl/Deployment/AssemblyLoader.cs | 105 +++ .../Impl/Deployment/AssemblyRequest.cs | 68 ++ .../Impl/Deployment/AssemblyRequestResult.cs | 80 ++ .../Impl/Deployment/GetAssemblyFunc.cs | 77 ++ .../Impl/Deployment/PeerAssemblyResolver.cs | 189 +++++ .../Impl/Deployment/PeerLoadingExtensions.cs | 65 ++ .../Impl/Deployment/PeerLoadingObjectHolder.cs | 90 ++ .../PeerLoadingObjectHolderSerializer.cs | 49 ++ modules/platforms/dotnet/Apache.Ignite.FxCop | 3 +- .../Impl/CacheFieldsQueryProvider.cs | 9 + .../Impl/CacheQueryExpressionVisitor.cs | 76 +- .../Apache.Ignite.Linq/Impl/ExpressionWalker.cs | 7 +- .../platforms/dotnet/Apache.Ignite/App.config | 11 + modules/spring/pom.xml | 7 + .../apache/ignite/cache/spring/SpringCache.java | 60 +- .../ignite/cache/spring/SpringCacheManager.java | 51 +- .../GridSpringCacheManagerMultiJvmSelfTest.java | 134 +++ .../GridSpringDynamicCacheTestService.java | 13 + .../ignite/cache/spring/SpringCacheTest.java | 2 +- .../ignite/cache/spring/spring-caching1.xml | 56 ++ .../ignite/cache/spring/spring-caching2.xml | 56 ++ modules/web-console/frontend/.babelrc | 9 +- modules/web-console/frontend/.gitignore | 1 - modules/web-console/frontend/app/app.js | 18 +- .../components/web-console-header/component.js | 6 +- .../app/modules/agent/AgentManager.service.js | 3 - .../frontend/gulpfile.babel.js/index.js | 26 - .../frontend/gulpfile.babel.js/paths.js | 64 -- .../frontend/gulpfile.babel.js/tasks/build.js | 21 - .../frontend/gulpfile.babel.js/tasks/bundle.js | 36 - .../frontend/gulpfile.babel.js/tasks/clean.js | 32 - .../frontend/gulpfile.babel.js/tasks/copy.js | 33 - .../gulpfile.babel.js/tasks/ignite-modules.js | 55 -- .../frontend/gulpfile.babel.js/tasks/watch.js | 30 - .../gulpfile.babel.js/webpack/common.js | 205 ----- .../webpack/environments/development.js | 60 -- .../webpack/environments/production.js | 41 - .../webpack/environments/test.js | 44 - .../frontend/gulpfile.babel.js/webpack/index.js | 36 - .../frontend/ignite_modules/index.js | 7 +- modules/web-console/frontend/package.json | 104 +-- .../frontend/test/karma.conf.babel.js | 13 +- modules/web-console/frontend/test/karma.conf.js | 2 +- .../frontend/webpack/webpack.common.js | 188 +++++ .../frontend/webpack/webpack.dev.babel.js | 100 +++ .../frontend/webpack/webpack.prod.babel.js | 64 ++ .../frontend/webpack/webpack.test.js | 33 + .../yardstick/IgniteBenchmarkArguments.java | 3 + .../cache/IgniteCacheAbstractBenchmark.java | 39 +- .../cache/IgnitePutObjectKeyBenchmark.java | 125 +++ .../cache/IgniteSqlQueryBenchmark.java | 2 +- .../IgniteSqlQueryDistributedJoinBenchmark.java | 2 +- .../cache/IgniteSqlQueryJoinBenchmark.java | 2 +- .../cache/IgniteSqlQueryPutBenchmark.java | 2 +- .../IgniteSqlQueryPutSeparatedBenchmark.java | 4 +- 339 files changed, 11869 insertions(+), 4214 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java ---------------------------------------------------------------------- diff --cc modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 669b7bc,6d05147..b04a969 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@@ -3061,7 -3064,7 +3064,7 @@@ public class IgniteKernal implements Ig try { checkClusterState(); - return ctx.cache().dynamicDestroyCache(cacheName, checkThreadTx, false); - return ctx.cache().dynamicDestroyCache(cacheName, sql, checkThreadTx); ++ return ctx.cache().dynamicDestroyCache(cacheName, sql, checkThreadTx, false); } finally { unguard(); http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupInfrastructure.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java ---------------------------------------------------------------------- diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 3e0f2c2,245e688..667b629 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@@ -760,20 -770,23 +772,21 @@@ public class GridCacheProcessor extend assert !ctx.config().isDaemon(); if (sharedCtx.pageStore() != null && sharedCtx.database().persistenceEnabled()) { - Set savedCacheNames = sharedCtx.pageStore().savedCacheNames(); + Map ccfgs = sharedCtx.pageStore().readCacheConfigurations(); - savedCacheNames.removeAll(caches.keySet()); + for (String cache : caches.keySet()) + ccfgs.remove(cache); - savedCacheNames.removeAll(internalCaches); + for (String cache : internalCaches) + ccfgs.remove(cache); - if (!F.isEmpty(savedCacheNames)) { + if (!F.isEmpty(ccfgs)) { if (log.isInfoEnabled()) - log.info("Register persistent caches: " + savedCacheNames); + log.info("Register persistent caches: " + ccfgs.keySet()); - for (String name : savedCacheNames) { - CacheConfiguration cfg = sharedCtx.pageStore().readConfiguration(name); - - // TODO IGNITE-5306 - set correct SQL flag below. - if (cfg != null) - addCacheOnJoin(cfg, false, caches, templates); - } ++ // TODO IGNITE-5306 - set correct SQL flag below. + for (CacheConfiguration ccfg : ccfgs.values()) - addCacheOnJoin(ccfg, caches, templates); ++ addCacheOnJoin(ccfg, false, caches, templates); } } } @@@ -1180,10 -1192,11 +1193,11 @@@ * @throws IgniteCheckedException If failed to start cache. */ @SuppressWarnings({"TypeMayBeWeakened", "unchecked"}) - private void startCache(GridCacheAdapter cache, QuerySchema schema) throws IgniteCheckedException { + private void startCache(CacheGroupDescriptor grpDesc, GridCacheAdapter cache, QuerySchema schema) throws IgniteCheckedException { GridCacheContext cacheCtx = cache.context(); - ctx.continuous().onCacheStart(cacheCtx); + if (sharedCtx.pageStore() != null) + sharedCtx.pageStore().initializeForCache(cacheCtx.config()); CacheConfiguration cfg = cacheCtx.config(); @@@ -1284,11 -1304,9 +1300,11 @@@ ctx.kernalContext().cache().context().database().onCacheStop(ctx); + ctx.kernalContext().cache().context().snapshot().onCacheStop(ctx); + ctx.group().stopCache(ctx, destroy); - U.stopLifecycleAware(log, lifecycleAwares(cache.configuration(), ctx.store().configuredStore())); + U.stopLifecycleAware(log, lifecycleAwares(ctx.group(), cache.configuration(), ctx.store().configuredStore())); if (log.isInfoEnabled()) { if (ctx.group().sharedGroup()) @@@ -2575,18 -2544,14 +2617,18 @@@ * @param checkThreadTx If {@code true} checks that current thread does not have active transactions. * @return Future that will be completed when cache is destroyed. */ - public IgniteInternalFuture dynamicDestroyCache(String cacheName, boolean checkThreadTx, boolean restart) { - public IgniteInternalFuture dynamicDestroyCache(String cacheName, boolean sql, boolean checkThreadTx) { ++ public IgniteInternalFuture dynamicDestroyCache(String cacheName, boolean sql, boolean checkThreadTx, boolean restart) { assert cacheName != null; if (checkThreadTx) checkEmptyTransactions(); - DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx, cacheName, true); + DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx, cacheName, sql, true); + req.stop(true); + req.destroy(true); + req.restart(restart); + return F.first(initiateCacheChanges(F.asList(req), false)); } @@@ -2603,12 -2567,8 +2645,12 @@@ List reqs = new ArrayList<>(cacheNames.size()); for (String cacheName : cacheNames) { - DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx, cacheName, true); + DynamicCacheChangeRequest req = DynamicCacheChangeRequest.stopRequest(ctx, cacheName, false, true); + req.stop(true); + req.destroy(true); + req.restart(restart); + reqs.add(req); } http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java ---------------------------------------------------------------------- diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java index f9f318a,78216db..c8dc8dc --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java @@@ -2366,7 -2359,7 +2366,7 @@@ public class IgniteCacheProxy ext IgniteInternalFuture fut; try { - fut = ctx.kernalContext().cache().dynamicDestroyCache(ctx.name(), true, false); - fut = ctx.kernalContext().cache().dynamicDestroyCache(ctx.name(), false, true); ++ fut = ctx.kernalContext().cache().dynamicDestroyCache(ctx.name(), false, true, false); } finally { onLeave(gate); http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java ---------------------------------------------------------------------- diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java index 76c7606,65a20eb..7c5c3b4 --- 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 @@@ -1249,9 -1145,18 +1256,21 @@@ public class GridDhtPartitionsExchangeF @Override public boolean onDone(@Nullable AffinityTopologyVersion res, @Nullable Throwable err) { boolean realExchange = !dummy && !forcePreload; + if (!done.compareAndSet(false, true)) + return dummy; + + if (err == null && + realExchange && + !cctx.kernalContext().clientNode() && + (serverNodeDiscoveryEvent() || affChangeMsg != null)) { + for (GridCacheContext cacheCtx : cctx.cacheContexts()) { + if (!cacheCtx.affinityNode() || cacheCtx.isLocal()) + continue; + + cacheCtx.continuousQueries().flushBackupQueue(exchId.topologyVersion()); + } + } + if (err == null && realExchange) { for (CacheGroupInfrastructure grp : cctx.cache().cacheGroups()) { if (grp.isLocal()) @@@ -1703,20 -1544,10 +1722,20 @@@ continue; CounterWithNodes maxCntr = maxCntrs.get(part.id()); - + - if (maxCntr == null || part.initialUpdateCounter() > maxCntr.cnt) - maxCntrs.put(part.id(), new CounterWithNodes(part.updateCounter(), cctx.localNodeId())); - else if (part.initialUpdateCounter() == maxCntr.cnt) + if (maxCntr == null && cntr == 0) { + CounterWithNodes cntrObj = new CounterWithNodes(cntr, cctx.localNodeId()); + + for (UUID nodeId : msgs.keySet()) { + if (top.partitionState(nodeId, part.id()) == GridDhtPartitionState.OWNING) + cntrObj.nodes.add(nodeId); + } + + maxCntrs.put(part.id(), cntrObj); + } + else if (maxCntr == null || cntr > maxCntr.cnt) + maxCntrs.put(part.id(), new CounterWithNodes(cntr, cctx.localNodeId())); + else if (cntr == maxCntr.cnt) maxCntr.nodes.add(cctx.localNodeId()); } http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheGroupsTest.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java ---------------------------------------------------------------------- diff --cc modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java index 4afaea3,ea3b9e2..4f606c9 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java @@@ -198,7 -198,7 +198,7 @@@ public class IgniteDynamicCacheStartSel GridTestUtils.runMultiThreaded(new Callable() { @Override public Object call() throws Exception { - futs.add(kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true, false)); - futs.add(kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, false, true)); ++ futs.add(kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, false, true, false)); return null; } @@@ -266,7 -266,7 +266,7 @@@ @Override public Object call() throws Exception { IgniteEx kernal = grid(ThreadLocalRandom.current().nextInt(nodeCount())); - futs.add(kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true, false)); - futs.add(kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, false, true)); ++ futs.add(kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, false, true, false)); return null; } http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsTestSuite.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/1305ce9c/modules/web-console/frontend/app/modules/agent/AgentManager.service.js ---------------------------------------------------------------------- diff --cc modules/web-console/frontend/app/modules/agent/AgentManager.service.js index a7196f1,3344ef2..0e6c3b8 --- a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js +++ b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js @@@ -15,11 -15,10 +15,8 @@@ * limitations under the License. */ - import io from 'socket.io-client'; // eslint-disable-line no-unused-vars - import maskNull from 'app/core/utils/maskNull'; - import { BehaviorSubject } from 'rxjs/BehaviorSubject'; -const maskNull = (val) => _.isNil(val) ? 'null' : val; - const State = { INIT: 'INIT', AGENT_DISCONNECTED: 'AGENT_DISCONNECTED',