You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by mm...@apache.org on 2020/12/02 19:04:46 UTC

[ignite] branch ignite-2.9.1 updated (fc22988 -> 02f3f69)

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

mmuzaf pushed a change to branch ignite-2.9.1
in repository https://gitbox.apache.org/repos/asf/ignite.git.


    from fc22988  IGNITE-13765 Incorrect work of predicates (< and >) in where clause with compound primary key (closes #8510)
     new 382d401  IGNITE-13553 Check type of the indexed fields added (#8330)
     new 9e578d8  IGNITE-12350: MVCC activated and causing memory leak (OOM) despite no mvccEnabled caches. This closes #8152
     new 102548a  IGNITE-13376 fix fields sequence at the PK (#8172)
     new 64b17bb  IGNITE-13280 fix improper index usage (#8067)
     new af6cb89  IGNITE-13379 Exception occur on SQL caches when client reconnect - Fixes #8178
     new 4d75a7d  IGNITE-13072 Synchronization problems when different classloaders are used for deployment of same class. - Fixes #7847.
     new 1eacfd6  IGNITE-13382 DurableBackgroundTask can abandon incomplete task - Fixes #8182.
     new 35dc1ae  IGNITE-13296 .NET: Fix transaction disposal and finalization
     new 17f47c2  IGNITE-13408 BinaryMetatadatView introduced (#8228)
     new 562733b  IGNITE-13409 MetastorageView and DistributedMetastorageView introduced. (#8225)
     new 8a007ba  IGNITE-13401 Java thin client: Fix unmarshalling failure when cache configuration message or binary type message start with byte 103 - Fixes #8216.
     new 33ce78e  IGNITE-13403 Update JDBC metadata to match actual capabilities (#8217)
     new ee064ba  IGNITE-13439 Printing detailed classpath slowdowns node initialization - Fixes #8246.
     new 083c959  IGNITE-13458 Added RebalancingPartitionsTotal metric. - Fixes #8254.
     new 278cbba  IGNITE-13484: Fixed ODBC example for several nodes. Added affinity key.
     new 4315c12  IGNITE-11312 Fix JDBC Driver properties for UI tools like DBeaver - Fixes #6900.
     new 6e20422  IGNITE-13491 Fix incorrect topology snapshot logger output about coordinator change. Fix remarks. - Fixes #8289.
     new 59bfe3f  IGNITE-13479 Control.sh script should use its own system variable for setting JVM options instead of JVM_OPTS - Fixes #8275.
     new 0958ab5  IGNITE-13495: fix ZookeeperDiscoveryImpl#getCoordinator() (#8313)
     new 7275d13  IGNITE-13304 Added javadocs for classes related to cache configuration enrichment. Fixes #8086
     new 3f13b99  IGNITE-13417 Fixed deserialization of CacheInterceptors on client nodes. Fixes #8230
     new e219b71  IGNITE-13540 Fix: exchange worker, waiting for new task from queue, considered as blocked. (#8324)
     new 909f172  IGNITE-13431 Fix NPE when PRIMITIVE strategy is used with BLOB value. (#8277)
     new 16c1688  IGNITE-13536 .NET: Document zombie child processes issue
     new a6e5f1e  IGNITE-13575 Fix invalid blocking thread reporting waiting on selector.select. Fix infinite loop while only one thread is registered in WorkersRegistry. (#8354)
     new caefd65  IGNITE-13563 .NET: Fix deserializing IBinaryObject containing an IBinaryObject field
     new e2d0742  IGNITE-13418 Fixed deadlock on multiple cache delete (#8229)
     new 6297b51  IGNITE-13569 disable archiving + walCompactionEnabled probably broke reading from wal on server restart - Fixes #8344.
     new 6df883a  IGNITE-13398 Avoid NPE when destroying the cache if there is a running service - Fixes #8376.
     new ac8dcc2  IGNITE-12794 Fix "Unexpected row key" assertion during scan query - Fixes #7541.
     new 0832093  IGNITE-13291 Add path validation to Ignite codebase. Remove unnecessary dependency to Apache Curator. (#8080)
     new 600d841  IGNITE-13577 Graceful node shutdown for Zookeeper Discovery SPI - Fixes #8371.
     new 47c276f1 IGNITE-13627 Metric registry remove not thread safe. (#8410)
     new 3c2a223  IGNITE-13665 When system worker is blocked, output its stack trace - Fixes #8442.
     new b664e1e  IGNITE-13719 Java thin client: Fix timeout on idle connection - Fixes #8480.
     new 0aebd00  IGNITE-13245 Rebalance future might hangs in no final state though all partitions are owned - Fixes #8025
     new 840e6ba  IGNITE-12320 Partial index rebuild fails in case indexed cache contains different datatypes - Fixes #8018
     new e6bb1c0  IGNITE-13251 Fixed deadlock between grid-timeout-worker and a thread opening a communication connection. Fixes #8033
     new 62efe7a  IGNITE-13235 Fix of deadlock in IgniteServiceProcessor. - Fixes #8009.
     new 7fa619b  IGNITE-13293 .NET: Fix enum serialization performance
     new 80da89b  IGNITE-13302 Fixed partition divergence in the system cache that may occur due to java thin client connecting. Fixes #8096
     new 25d3f35  IGNITE-13102 IgniteCache#isClosed() returns false on server node even if the cache had been closed before - Fixes #8087.
     new e99ba03  IGNITE-12806 Fixed incorret usage of Class.isAssignableFrom in SystemViewLocal and SystemViewMBean (#7592)
     new 487bce8  IGNITE-13402 Fixed AssertionError during expiration cache entries. Fixes #8215
     new 891c0bf  IGNITE-13427 Fix Metastore view iteration on unmarshalleable keys (#8240)
     new 51e8947  IGNITE-9474 Fix incorrect cache.size() with ttl enabled (#8391)
     new 8795574  IGNITE-13174 C++: Add CMake support for Windows
     new e7bb6ff  Style fixes
     new afe6e41  Ignite 2.9.1 release notes
     new 8ade00b  IGNITE-13643 : Disable socket linger dy default in Tcp Discovery Spi (#8407)
     new a23806a  Ignite 2.9.1 version update
     new aad4933  Updated DEB & RPM package version to 2.9.1
     new 1d93aa6  Versions update
     new be8448a  IGNITE-13699 Register ZookeeperDiscovery metrics to new metrics framework (#8458)
     new 80d364e  updated release notes
     new a7adb31  IGNITE-13572 Don't skip filtering for caches with zero backups - Fixes #8439.
     new c4a9e05  IGNITE-13770 Fix NPE in Ignite.dataRegionMetrics with empty persistent region - Fixes #8506.
     new 02f3f69  IGNITE-13388 Fix apache-ignite deb package dependency on JVM package - Fixes #8191.

The 58 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 RELEASE_NOTES.txt                                  |  59 ++++
 bin/control.bat                                    |  50 ++--
 bin/control.sh                                     |  41 +--
 bin/ignite.bat                                     |   9 +-
 docs/_docs/net-specific/net-troubleshooting.adoc   | 161 ++++++++++
 examples/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 +-
 modules/camel/pom.xml                              |   2 +-
 modules/cassandra/pom.xml                          |   2 +-
 modules/cassandra/serializers/pom.xml              |   4 +-
 modules/cassandra/store/pom.xml                    |   4 +-
 .../cassandra/persistence/PersistenceSettings.java |   4 +-
 .../tests/IgnitePersistentStorePrimitiveTest.java  | 142 +++++++++
 .../store/IgniteCassandraStoreTestSuite.java       |   4 +-
 modules/clients/pom.xml                            |   2 +-
 .../internal/jdbc2/JdbcMetadataSelfTest.java       |  19 +-
 .../apache/ignite/jdbc/JdbcMetadataSelfTest.java   |  16 +
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java      |   2 +
 .../thin/JdbcThinConnectionPropertiesTest.java     |  64 ++++
 .../ignite/jdbc/thin/JdbcThinMetadataSelfTest.java |  30 +-
 modules/cloud/pom.xml                              |   2 +-
 modules/codegen/pom.xml                            |   2 +-
 .../SystemViewRowAttributeWalkerGenerator.java     |   4 +
 modules/compatibility/pom.xml                      |   2 +-
 modules/compress/pom.xml                           |   2 +-
 modules/core/pom.xml                               |   2 +-
 .../java/org/apache/ignite/cache/QueryEntity.java  |   3 +-
 .../ignite/configuration/CacheConfiguration.java   |   1 +
 .../configuration/NearCacheConfiguration.java      |   1 -
 .../ignite/internal/GridKernalGatewayImpl.java     |   7 +-
 .../org/apache/ignite/internal/IgniteFeatures.java |  37 ++-
 .../org/apache/ignite/internal/IgniteKernal.java   |  21 +-
 .../org/apache/ignite/internal/IgnitionEx.java     |  14 +-
 .../client/thin/ClientCacheAffinityMapping.java    |   2 +-
 .../client/thin/ClientClusterGroupImpl.java        |  23 +-
 .../internal/client/thin/ClientClusterImpl.java    |   5 +-
 .../ignite/internal/client/thin/ClientUtils.java   |  21 +-
 .../internal/client/thin/TcpClientChannel.java     |  23 +-
 .../internal/client/thin/TcpIgniteClient.java      |   2 +-
 .../internal/commandline/CommandHandler.java       |   8 +-
 .../ignite/internal/jdbc/JdbcDatabaseMetadata.java |   8 +-
 .../jdbc/thin/ConnectionPropertiesImpl.java        |   2 +-
 .../jdbc/thin/JdbcThinDatabaseMetadata.java        |   6 +-
 .../internal/jdbc2/JdbcDatabaseMetadata.java       |   8 +-
 .../deployment/GridDeploymentLocalStore.java       |  67 +++--
 .../managers/discovery/GridDiscoveryManager.java   |  11 +-
 .../walker/BinaryMetadataViewWalker.java           |  56 ++++
 .../systemview/walker/MetastorageViewWalker.java   |  46 +++
 .../cache/CacheConfigurationEnricher.java          |  58 ++--
 .../cache/CacheConfigurationEnrichment.java        |  41 +--
 .../cache/CacheConfigurationSplitter.java          |  31 +-
 .../cache/CacheConfigurationSplitterImpl.java      |  52 +++-
 .../cache/CacheConfigurationSplitterOldFormat.java |   7 +-
 .../internal/processors/cache/CacheData.java       |   3 +-
 .../internal/processors/cache/CacheGroupData.java  |   1 +
 .../processors/cache/CacheGroupDescriptor.java     |   1 +
 .../processors/cache/ClusterCachesInfo.java        |   7 +-
 .../processors/cache/DynamicCacheDescriptor.java   |   1 +
 .../processors/cache/GridCacheAttributes.java      |  15 +-
 .../processors/cache/GridCacheMapEntry.java        |  22 +-
 .../cache/GridCachePartitionExchangeManager.java   |  47 +--
 .../processors/cache/GridCacheProcessor.java       |  10 +-
 .../cache/GridCacheSharedTtlCleanupManager.java    |  11 +
 .../cache/IgniteCacheOffheapManager.java           |   6 +
 .../cache/IgniteCacheOffheapManagerImpl.java       |  26 +-
 .../processors/cache/IgniteCacheProxyImpl.java     |   2 +-
 .../internal/processors/cache/StoredCacheData.java |  13 +-
 .../binary/CacheObjectBinaryProcessorImpl.java     |  12 +
 .../dht/preloader/GridDhtPartitionDemander.java    |  13 +-
 .../cache/distributed/near/GridNearTxLocal.java    |  57 ++--
 .../processors/cache/mvcc/MvccProcessorImpl.java   |  36 ++-
 .../GridCacheDatabaseSharedManager.java            |  40 +++
 .../cache/persistence/metastorage/MetaStorage.java |   5 +
 .../pendingtask/DurableBackgroundTask.java         |  17 ++
 .../cache/persistence/pagemem/PageMemoryImpl.java  |   4 +-
 .../persistence/wal/FileWriteAheadLogManager.java  |  36 ++-
 .../cache/transactions/IgniteTxManager.java        |  18 +-
 .../TransactionProxyRollbackOnlyImpl.java          |   2 +-
 .../processors/cache/tree/PendingEntriesTree.java  |   2 +-
 .../localtask/DurableBackgroundTasksProcessor.java |  35 ++-
 .../persistence/DistributedMetaStorageImpl.java    |  40 ++-
 .../processors/metric/GridMetricManager.java       |  36 ++-
 .../transactions/PlatformTransactions.java         |   8 +
 .../processors/query/GridQueryProcessor.java       |  57 +---
 .../processors/query/GridQueryTypeDescriptor.java  |  13 +
 .../processors/query/IgniteSQLException.java       |   4 +-
 .../processors/query/QueryTypeDescriptorImpl.java  |  80 +++++
 .../internal/processors/query/QueryUtils.java      |  17 +-
 .../schema/SchemaIndexCachePartitionWorker.java    |  13 +-
 .../query/schema/SchemaIndexCacheVisitorImpl.java  |   8 +-
 .../processors/service/IgniteServiceProcessor.java |  13 +-
 .../apache/ignite/internal/util/IgniteUtils.java   |  16 -
 .../ignite/internal/util/nio/GridNioServer.java    |  21 +-
 .../ignite/internal/worker/WorkersRegistry.java    |   4 +-
 .../ignite/spi/discovery/tcp/TcpDiscoverySpi.java  |   2 +-
 .../spi/indexing/IndexingQueryFilterImpl.java      |   4 -
 .../ignite/spi/systemview/jmx/SystemViewMBean.java |  65 ++--
 .../spi/systemview/view/BinaryMetadataView.java    |  85 ++++++
 .../spi/systemview/view/MetastorageView.java}      |  41 ++-
 .../main/resources/META-INF/classnames.properties  |   1 +
 modules/core/src/main/resources/ignite.properties  |   2 +-
 .../apache/ignite/ClassPathContentLoggingTest.java |  17 +-
 .../client/ClientCacheConfigurationTest.java       |  56 +++-
 .../org/apache/ignite/client/IgniteBinaryTest.java |  25 ++
 .../failure/ExchangeWorkerWaitingForTaskTest.java  |  77 +++++
 .../ignite/failure/SystemWorkersBlockingTest.java  | 130 ++++++--
 .../RaceOnDeployClassesWithSameAliases.java        | 147 +++++++++
 .../client/thin/AbstractThinClientTest.java        |  20 +-
 .../ignite/internal/client/thin/TimeoutTest.java   | 220 ++++++++++++++
 .../managers/IgniteDiagnosticMessagesTest.java     |  56 ++++
 .../IgniteTopologyPrintFormatSelfTest.java         |  27 +-
 .../ignite/internal/metric/JmxExporterSpiTest.java | 120 ++++++++
 .../ignite/internal/metric/MetricsSelfTest.java    |  89 ++++++
 .../ignite/internal/metric/SystemViewSelfTest.java | 116 ++++++++
 .../cache/CacheGroupsMetricsRebalanceTest.java     |  12 +
 .../cache/CacheStopAndDestroySelfTest.java         |  22 ++
 .../GridCacheInterceptorAbstractSelfTest.java      |  13 +-
 .../IgniteCacheClusterReadOnlyModeSelfTest.java    |   5 -
 .../NoPresentCacheInterceptorOnClientTest.java     | 203 +++++++++++++
 .../SysCacheInconsistencyInternalKeyTest.java      |  89 ++++++
 .../cache/WalModeChangeAdvancedSelfTest.java       |   4 +-
 ...lanceIsProcessingWhenAssignmentIsEmptyTest.java | 157 ++++++++++
 .../cache/mvcc/MvccStructuresOverheadTest.java     | 128 ++++++++
 .../processors/cache/persistence/EagerTtlTest.java | 203 +++++++++++++
 .../db/IgnitePdsDataRegionMetricsTest.java         |   7 +
 .../cache/persistence/db/RebalanceBlockingSPI.java | 102 +++++++
 .../db/wal/WalCompactionNoArchiverTest.java        | 135 +++++++++
 .../ScanQueryConcurrentUpdatesAbstractTest.java    | 209 +++++++++++++
 .../query/ScanQueryConcurrentUpdatesTest.java      |  53 ++++
 .../testframework/junits/GridAbstractTest.java     |   3 +
 .../ignite/testsuites/IgniteBasicTestSuite.java    |   2 +
 .../testsuites/IgniteCacheMvccTestSuite.java       |   4 +-
 .../testsuites/IgniteCacheMvccTestSuite2.java      |   2 +
 .../ignite/testsuites/IgniteCacheTestSuite2.java   |   4 +
 .../ignite/testsuites/IgniteCacheTestSuite6.java   |   3 +
 .../ignite/testsuites/IgniteP2PSelfTestSuite.java  |   2 +
 .../ignite/testsuites/IgnitePdsMvccTestSuite2.java |   2 +
 .../ignite/testsuites/IgnitePdsMvccTestSuite3.java |  24 +-
 .../ignite/testsuites/IgnitePdsTestSuite2.java     |   2 +
 .../ignite/testsuites/IgnitePdsTestSuite3.java     |   2 +
 modules/dev-utils/ignite-modules-test/build.gradle |   2 +-
 modules/dev-utils/pom.xml                          |   2 +-
 modules/direct-io/pom.xml                          |   2 +-
 modules/extdata/p2p/pom.xml                        |   2 +-
 .../org/apache/ignite/tests/p2p/cache/Address.java |  57 ++++
 .../apache/ignite/tests/p2p/cache/Container.java}  |  27 +-
 .../tests/p2p/cache/OddEvenCacheInterceptor.java   |  58 ++++
 .../ignite/tests/p2p/cache/Organization.java}      |  51 ++--
 modules/extdata/platform/pom.xml                   |   2 +-
 modules/extdata/uri/modules/uri-dependency/pom.xml |   2 +-
 modules/extdata/uri/pom.xml                        |   2 +-
 modules/gce/pom.xml                                |   2 +-
 modules/geospatial/pom.xml                         |   2 +-
 modules/hibernate-4.2/pom.xml                      |   2 +-
 modules/hibernate-5.1/pom.xml                      |   2 +-
 modules/hibernate-5.3/pom.xml                      |   2 +-
 modules/hibernate-core/pom.xml                     |   2 +-
 modules/ignored-tests/pom.xml                      |   2 +-
 modules/indexing/pom.xml                           |   2 +-
 .../processors/query/h2/CommandProcessor.java      |   2 +-
 .../h2/DurableBackgroundCleanupIndexTreeTask.java  |  19 ++
 .../processors/query/h2/H2TableDescriptor.java     |  23 +-
 .../processors/query/h2/IgniteH2Indexing.java      |   4 +-
 .../processors/query/h2/IndexBuildClosure.java     |  54 ----
 .../query/h2/IndexRebuildPartialClosure.java       |  24 +-
 .../processors/query/h2/SchemaManager.java         |   6 +-
 .../processors/query/h2/database/H2TreeIndex.java  |  70 ++---
 .../query/h2/database/H2TreeIndexBase.java         |   2 +-
 .../processors/query/h2/opt/GridH2IndexBase.java   |  14 +-
 .../processors/query/h2/opt/GridH2ProxyIndex.java  |   8 +-
 .../processors/query/h2/opt/GridH2Table.java       |  12 +-
 .../processors/query/h2/opt/H2IndexCostedBase.java | 234 +++++++++++++++
 .../ignite/spi/systemview/SystemViewLocal.java     | 122 ++++----
 .../org/apache/ignite/client/ClientTestSuite.java  |   4 +-
 .../cache/index/AbstractIndexingCommonTest.java    |   2 +-
 .../processors/cache/index/BasicIndexTest.java     | 236 +++++++++++++--
 .../ClientReconnectWithSqlTableConfiguredTest.java | 166 +++++++++++
 .../cache/index/SqlPartitionEvictionTest.java      | 215 ++++++++++++++
 .../cache/metric/SqlViewExporterSpiTest.java       | 111 ++++++-
 .../db/LongDestroyDurableBackgroundTaskTest.java   | 208 ++++++++++++-
 .../MultipleParallelCacheDeleteDeadlockTest.java   | 328 +++++++++++++++++++++
 .../query/ScanQueryConcurrentSqlUpdatesTest.java   |  86 ++++++
 .../processors/cache/ttl/CacheSizeTtlTest.java     | 130 ++++++++
 .../query/CreateIndexOnInvalidDataTypeTest.java    |  18 +-
 .../processors/query/SqlSystemViewsSelfTest.java   |   2 +-
 .../query/WrongQueryEntityFieldTypeTest.java       | 289 ++++++++++++++++++
 ...lRebuildTest.java => GridIndexRebuildTest.java} | 198 +++++++++++--
 .../query/h2/IgniteSqlQueryMinMaxTest.java         |   5 +-
 .../IgniteBinaryCacheQueryTestSuite.java           |  12 +-
 .../IgniteBinaryCacheQueryTestSuite2.java          |   2 +
 .../IgniteCacheWithIndexingTestSuite.java          |   2 +
 .../testsuites/IgnitePdsWithIndexingTestSuite.java |   6 +-
 modules/jcl/pom.xml                                |   2 +-
 modules/jms11/pom.xml                              |   2 +-
 modules/jta/pom.xml                                |   2 +-
 modules/kafka/pom.xml                              |   2 +-
 modules/kubernetes/pom.xml                         |   2 +-
 modules/log4j/pom.xml                              |   2 +-
 modules/log4j2/pom.xml                             |   2 +-
 modules/mesos/pom.xml                              |   2 +-
 modules/ml/h2o-model-parser/pom.xml                |   2 +-
 modules/ml/pom.xml                                 |   2 +-
 modules/ml/spark-model-parser/pom.xml              |   2 +-
 modules/ml/xgboost-model-parser/pom.xml            |   2 +-
 modules/mqtt/pom.xml                               |   2 +-
 modules/opencensus/pom.xml                         |   2 +-
 modules/osgi-karaf/pom.xml                         |   2 +-
 modules/osgi-paxlogging/pom.xml                    |   2 +-
 modules/osgi/pom.xml                               |   2 +-
 modules/platforms/cpp/CMakeLists.txt               |   6 +-
 modules/platforms/cpp/binary/CMakeLists.txt        |   2 +
 .../include/ignite/impl/binary/binary_field_meta.h |   4 +-
 .../ignite/impl/binary/binary_object_impl.h        |   2 +-
 .../include/ignite/impl/binary/binary_type_impl.h  |   2 +-
 .../ignite/impl/binary/binary_type_snapshot.h      |   4 +-
 .../binary/src/impl/binary/binary_field_meta.cpp   |   4 +-
 .../binary/src/impl/binary/binary_object_impl.cpp  |   2 +-
 .../src/impl/binary/binary_type_snapshot.cpp       |   4 +-
 modules/platforms/cpp/cmake/FindIgnite.cmake       |  27 +-
 modules/platforms/cpp/cmake/FindODBC.cmake         |   9 +-
 modules/platforms/cpp/common/CMakeLists.txt        |   2 +
 .../cpp/common/include/ignite/common/big_integer.h |  14 +-
 .../cpp/common/include/ignite/common/decimal.h     |  14 +-
 .../common/include/ignite/common/platform_utils.h  |   2 +-
 .../cpp/common/include/ignite/common/utils.h       |  12 +-
 .../common/os/linux/src/common/platform_utils.cpp  |   2 +-
 .../os/win/include/ignite/common/concurrent_os.h   |   2 +-
 .../common/os/win/src/common/platform_utils.cpp    |   2 +-
 modules/platforms/cpp/common/src/common/utils.cpp  |  14 +-
 modules/platforms/cpp/core-test/CMakeLists.txt     |   4 +-
 modules/platforms/cpp/core/CMakeLists.txt          |   4 +
 modules/platforms/cpp/examples/CMakeLists.txt      |   2 +
 .../cpp/examples/compute-example/CMakeLists.txt    |   3 +-
 .../continuous-query-example/CMakeLists.txt        |   3 +-
 .../cpp/examples/odbc-example/CMakeLists.txt       |   8 +-
 .../examples/odbc-example/config/example-odbc.xml  |  19 +-
 .../cpp/examples/odbc-example/src/odbc_example.cpp |  98 +++---
 .../cpp/examples/put-get-example/CMakeLists.txt    |   3 +-
 .../cpp/examples/query-example/CMakeLists.txt      |   3 +-
 .../examples/query-example/src/query_example.cpp   |  48 ++-
 .../thin-client-put-get-example/CMakeLists.txt     |   3 +-
 modules/platforms/cpp/jni/CMakeLists.txt           |  10 +-
 modules/platforms/cpp/network/CMakeLists.txt       |   5 +-
 .../cpp/network/include/ignite/network/network.h   |   7 +-
 .../cpp/network/include/ignite/network/utils.h     |   4 +-
 .../cpp/network/os/win/src/network/utils.cpp       |   2 +-
 .../platforms/cpp/network/src/network/network.cpp  |   6 +-
 modules/platforms/cpp/odbc-test/CMakeLists.txt     |   4 +-
 modules/platforms/cpp/odbc/CMakeLists.txt          |  12 +-
 .../cpp/odbc/install/ignite-odbc-amd64.wxs         |   2 +-
 .../platforms/cpp/odbc/install/ignite-odbc-x86.wxs |   2 +-
 modules/platforms/cpp/thin-client/CMakeLists.txt   |   4 +
 .../Properties/AssemblyInfo.cs                     |   6 +-
 .../Properties/AssemblyInfo.cs                     |   6 +-
 .../{Program.cs => Models/BenchmarkEnum.cs}        |  23 +-
 .../{Program.cs => Models/ClassWithEnumField.cs}   |  21 +-
 .../{Program.cs => Models/ClassWithIntField.cs}    |  22 +-
 .../Apache.Ignite.BenchmarkDotNet/Program.cs       |   2 +-
 .../ThinClient/ThinClientCachePutBenchmark.cs      | 105 +++++++
 .../Properties/AssemblyInfo.cs                     |   6 +-
 .../Properties/AssemblyInfo.cs                     |   4 +-
 .../Properties/AssemblyInfo.cs                     |   6 +-
 .../Properties/AssemblyInfo.cs                     |   6 +-
 .../Apache.Ignite.Core.Tests.csproj                |   2 +
 .../Binary/BinarySelfTest.cs                       |  27 ++
 .../Apache.Ignite.Core.Tests/Binary/EnumsTest.cs   |   6 +
 .../Cache/CacheLocalActiveTransactionsTest.cs      | 208 +++++++++++++
 .../Cache/CacheTransactionGridStopTest.cs}         |  30 +-
 .../Cache/OptimisticTransactionTest.cs             |   9 +-
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs     |   2 +-
 .../IgniteStartStopTest.cs                         |   2 +-
 .../Properties/AssemblyInfo.cs                     |   6 +-
 .../Apache.Ignite.Core/Apache.Ignite.Core.csproj   |   2 +
 .../Common/IgniteIllegalStateException.cs          |  69 +++++
 .../Apache.Ignite.Core/Impl/Binary/BinaryObject.cs |   2 +-
 .../Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs  |  74 +++++
 .../Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs |  35 ++-
 .../Impl/Binary/Metadata/BinaryType.cs             |  55 +---
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs      |   1 +
 .../Impl/Transactions/TransactionImpl.cs           |  12 +-
 .../Transactions/TransactionRollbackOnlyProxy.cs   | 261 ++++++++++++++++
 .../Impl/Transactions/TransactionsImpl.cs          |  55 +++-
 .../Apache.Ignite.Core/Properties/AssemblyInfo.cs  |   6 +-
 .../Transactions/ITransactions.cs                  |  19 +-
 .../Properties/AssemblyInfo.cs                     |   6 +-
 .../Properties/AssemblyInfo.cs                     |   6 +-
 .../Apache.Ignite.Linq/Properties/AssemblyInfo.cs  |   6 +-
 .../Properties/AssemblyInfo.cs                     |   6 +-
 .../Apache.Ignite.NLog/Properties/AssemblyInfo.cs  |   6 +-
 .../Apache.Ignite/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/slf4j/pom.xml                              |   2 +-
 modules/spark-2.4/pom.xml                          |   2 +-
 modules/spark/pom.xml                              |   2 +-
 modules/spring-data-2.0/pom.xml                    |   2 +-
 modules/spring-data-2.2/pom.xml                    |   2 +-
 modules/spring-data/pom.xml                        |   2 +-
 modules/spring/pom.xml                             |   2 +-
 modules/sqlline/pom.xml                            |   2 +-
 modules/ssh/pom.xml                                |   2 +-
 modules/storm/pom.xml                              |   2 +-
 modules/tools/pom.xml                              |   2 +-
 modules/urideploy/pom.xml                          |   2 +-
 modules/visor-console-2.10/pom.xml                 |   2 +-
 modules/visor-console/pom.xml                      |   2 +-
 modules/visor-plugins/pom.xml                      |   2 +-
 modules/web/ignite-appserver-test/pom.xml          |   2 +-
 modules/web/ignite-websphere-test/pom.xml          |   2 +-
 modules/web/pom.xml                                |   2 +-
 modules/yardstick/pom.xml                          |   2 +-
 modules/yarn/pom.xml                               |   2 +-
 modules/yarn/src/main/resources/ignite.properties  |   2 +-
 modules/zookeeper/pom.xml                          |   2 +-
 .../spi/discovery/zk/ZookeeperDiscoverySpi.java    |  20 +-
 ...stractCallabck.java => ZkAbstractCallback.java} |   6 +-
 .../zk/internal/ZkAbstractChildrenCallback.java    |   2 +-
 .../discovery/zk/internal/ZkAbstractWatcher.java   |   2 +-
 .../zk/internal/ZkDiscoveryEventData.java          |   4 +-
 ...ata.java => ZkDiscoveryNodeLeaveEventData.java} |  45 ++-
 .../spi/discovery/zk/internal/ZkIgnitePaths.java   | 120 +++++++-
 .../spi/discovery/zk/internal/ZkRunnable.java      |   2 +-
 .../spi/discovery/zk/internal/ZkRuntimeState.java  |   2 +-
 .../spi/discovery/zk/internal/ZookeeperClient.java |  26 ++
 .../zk/internal/ZookeeperDiscoveryImpl.java        | 160 +++++++---
 .../zk/internal/ZookeeperDiscoveryStatistics.java  |  53 +++-
 .../zk/ZookeeperDiscoverySpiTestSuite1.java        |   4 +
 ...erDiscoveryConcurrentStartAndStartStopTest.java |   6 +-
 .../zk/internal/ZookeeperDiscoveryMiscTest.java    |  49 ++-
 ...perDiscoveryRandomStopOrFailConcurrentTest.java | 264 +++++++++++++++++
 ...coverySegmentationAndConnectionRestoreTest.java |   4 +-
 .../zk/internal/ZookeeperDiscoverySpiTestBase.java |  90 ++++++
 .../internal/ZookeeperDiscoverySpiTestHelper.java  |   6 +-
 ...perDiscoveryTopologyChangeAndReconnectTest.java |  90 ------
 .../zk/internal/ZookeeperValidatePathsTest.java    | 126 ++++++++
 .../zookeeper/ZkTestClientCnxnSocketNIO.java       |   3 +-
 packaging/deb/changelog                            |   6 +
 packaging/deb/control                              |   2 +-
 packaging/rpm/apache-ignite.spec                   |   5 +-
 pom.xml                                            |   2 +-
 345 files changed, 8791 insertions(+), 1407 deletions(-)
 create mode 100644 docs/_docs/net-specific/net-troubleshooting.adoc
 create mode 100644 modules/cassandra/store/src/test/java/org/apache/ignite/tests/IgnitePersistentStorePrimitiveTest.java
 create mode 100644 modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionPropertiesTest.java
 create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/BinaryMetadataViewWalker.java
 create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/MetastorageViewWalker.java
 create mode 100644 modules/core/src/main/java/org/apache/ignite/spi/systemview/view/BinaryMetadataView.java
 copy modules/core/src/{test/java/org/apache/ignite/testsuites/IgnitePdsMvccTestSuite3.java => main/java/org/apache/ignite/spi/systemview/view/MetastorageView.java} (57%)
 create mode 100644 modules/core/src/test/java/org/apache/ignite/failure/ExchangeWorkerWaitingForTaskTest.java
 create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/RaceOnDeployClassesWithSameAliases.java
 create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/client/thin/TimeoutTest.java
 create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/NoPresentCacheInterceptorOnClientTest.java
 create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/SysCacheInconsistencyInternalKeyTest.java
 create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/RebalanceIsProcessingWhenAssignmentIsEmptyTest.java
 create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/MvccStructuresOverheadTest.java
 create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/EagerTtlTest.java
 create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/RebalanceBlockingSPI.java
 create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/WalCompactionNoArchiverTest.java
 create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/ScanQueryConcurrentUpdatesAbstractTest.java
 create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/ScanQueryConcurrentUpdatesTest.java
 create mode 100644 modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/Address.java
 copy modules/{platforms/cpp/network/include/ignite/network/utils.h => extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/Container.java} (66%)
 create mode 100644 modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/OddEvenCacheInterceptor.java
 copy modules/{platforms/cpp/network/include/ignite/network/utils.h => extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/cache/Organization.java} (61%)
 delete mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IndexBuildClosure.java
 create mode 100644 modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/H2IndexCostedBase.java
 create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/ClientReconnectWithSqlTableConfiguredTest.java
 create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/SqlPartitionEvictionTest.java
 create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/MultipleParallelCacheDeleteDeadlockTest.java
 create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/query/ScanQueryConcurrentSqlUpdatesTest.java
 create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ttl/CacheSizeTtlTest.java
 create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/WrongQueryEntityFieldTypeTest.java
 rename modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/{GridIndexFullRebuildTest.java => GridIndexRebuildTest.java} (58%)
 copy modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/{Program.cs => Models/BenchmarkEnum.cs} (68%)
 copy modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/{Program.cs => Models/ClassWithEnumField.cs} (68%)
 copy modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/{Program.cs => Models/ClassWithIntField.cs} (68%)
 create mode 100644 modules/platforms/dotnet/Apache.Ignite.BenchmarkDotNet/ThinClient/ThinClientCachePutBenchmark.cs
 create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheLocalActiveTransactionsTest.cs
 copy modules/platforms/dotnet/{Apache.Ignite.BenchmarkDotNet/Program.cs => Apache.Ignite.Core.Tests/Cache/CacheTransactionGridStopTest.cs} (51%)
 create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Common/IgniteIllegalStateException.cs
 create mode 100644 modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionRollbackOnlyProxy.cs
 rename modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/{ZkAbstractCallabck.java => ZkAbstractCallback.java} (92%)
 rename modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/{ZkDiscoveryNodeFailEventData.java => ZkDiscoveryNodeLeaveEventData.java} (53%)
 create mode 100644 modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryRandomStopOrFailConcurrentTest.java
 create mode 100644 modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperValidatePathsTest.java