You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/07/11 12:27:16 UTC

[1/8] ignite git commit: IGNITE-5479 - PersistentStoreConfiguration improvements

Repository: ignite
Updated Branches:
  refs/heads/ignite-5479 [created] 727df1871


IGNITE-5479 - PersistentStoreConfiguration improvements


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

Branch: refs/heads/ignite-5479
Commit: b3df5117a4cf764e6ccf8f23bd072df427f9bfbc
Parents: c52d2bf
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Jul 10 16:33:12 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Jul 10 16:33:12 2017 +0300

----------------------------------------------------------------------
 .../PersistentStoreConfiguration.java           | 47 +++++++++++---------
 .../GridCacheDatabaseSharedManager.java         | 10 ++---
 .../wal/FileWriteAheadLogManager.java           |  6 +--
 .../utils/PlatformConfigurationUtils.java       |  4 +-
 .../node/VisorPersistentStoreConfiguration.java | 28 ++++++------
 .../PersistentStoreConfiguration.cs             |  6 +--
 6 files changed, 53 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b3df5117/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
index 4792483..1d4e2ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
@@ -70,7 +70,7 @@ public class PersistentStoreConfiguration implements Serializable {
     public static final int DFLT_WAL_FLUSH_FREQ = 2000;
 
     /** Default wal fsync delay. */
-    public static final int DFLT_WAL_FSYNC_DELAY = 1;
+    public static final int DFLT_WAL_FSYNC_DELAY = 1000;
 
     /** Default wal record iterator buffer size. */
     public static final int DFLT_WAL_RECORD_ITERATOR_BUFFER_SIZE = 64 * 1024 * 1024;
@@ -90,8 +90,8 @@ public class PersistentStoreConfiguration implements Serializable {
     /** Checkpointing frequency. */
     private long checkpointingFreq = DFLT_CHECKPOINTING_FREQ;
 
-    /** Lock wait time. */
-    private int lockWaitTime = DFLT_LOCK_WAIT_TIME;
+    /** Lock wait time, in milliseconds. */
+    private long lockWaitTime = DFLT_LOCK_WAIT_TIME;
 
     /** */
     private Long checkpointingPageBufSize = DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE;
@@ -124,10 +124,10 @@ public class PersistentStoreConfiguration implements Serializable {
     private int tlbSize = DFLT_TLB_SIZE;
 
     /** Wal flush frequency in milliseconds. */
-    private int walFlushFreq = DFLT_WAL_FLUSH_FREQ;
+    private long walFlushFreq = DFLT_WAL_FLUSH_FREQ;
 
     /** Wal fsync delay. */
-    private int walFsyncDelay = DFLT_WAL_FSYNC_DELAY;
+    private long walFsyncDelay = DFLT_WAL_FSYNC_DELAY;
 
     /** Wal record iterator buffer size. */
     private int walRecordIterBuffSize = DFLT_WAL_RECORD_ITERATOR_BUFFER_SIZE;
@@ -243,21 +243,23 @@ public class PersistentStoreConfiguration implements Serializable {
     }
 
     /**
-     * Time out in second, while wait and try get file lock for start persist manager.
+     * Time out in milliseonds to wait when acquiring persistence store lock file before failing the
+     * local node.
      *
-     * @return Time for wait.
+     * @return Lock wait time in milliseconds.
      */
-    public int getLockWaitTime() {
+    public long getLockWaitTime() {
         return lockWaitTime;
     }
 
     /**
-     * Time out in milliseconds, while wait and try get file lock for start persist manager.
+     * Time out in milliseconds  to wait when acquiring persistence store lock file before failing the
+     * local node.
      *
-     * @param lockWaitTime Lock wait time.
+     * @param lockWaitTime Lock wait time in milliseconds.
      * @return {@code this} for chaining.
      */
-    public PersistentStoreConfiguration setLockWaitTime(int lockWaitTime) {
+    public PersistentStoreConfiguration setLockWaitTime(long lockWaitTime) {
         this.lockWaitTime = lockWaitTime;
 
         return this;
@@ -474,19 +476,22 @@ public class PersistentStoreConfiguration implements Serializable {
     }
 
     /**
-     *  Property define how often will be fsync, in milliseconds.
-     *  In background mode, exist thread which do fsync by timeout.
+     *  This property define how often WAL will be fsync-ed in {@code BACKGROUND} mode. Ignored for
+     *  all other WAL modes.
      *
-     * @return Flush frequency.
+     * @return WAL flush frequency, in milliseconds.
      */
-    public int getWalFlushFrequency() {
+    public long getWalFlushFrequency() {
         return walFlushFreq;
     }
 
     /**
-     * @param walFlushFreq Wal flush frequency, in milliseconds.
+     *  This property define how often WAL will be fsync-ed in {@code BACKGROUND} mode. Ignored for
+     *  all other WAL modes.
+     *
+     * @param walFlushFreq WAL flush frequency, in milliseconds.
      */
-    public PersistentStoreConfiguration setWalFlushFrequency(int walFlushFreq) {
+    public PersistentStoreConfiguration setWalFlushFrequency(long walFlushFreq) {
         this.walFlushFreq = walFlushFreq;
 
         return this;
@@ -495,15 +500,15 @@ public class PersistentStoreConfiguration implements Serializable {
     /**
      * Gets the fsync delay, in nanoseconds.
      */
-    public int getWalFsyncDelay() {
+    public long getWalFsyncDelayNanos() {
         return walFsyncDelay <= 0 ? DFLT_WAL_FSYNC_DELAY : walFsyncDelay;
     }
 
     /**
-     * @param walFsyncDelay Wal fsync delay, in nanoseconds.
+     * @param walFsyncDelayNanos Wal fsync delay, in nanoseconds.
      */
-    public PersistentStoreConfiguration setWalFsyncDelay(int walFsyncDelay) {
-        this.walFsyncDelay = walFsyncDelay;
+    public PersistentStoreConfiguration setWalFsyncDelayNanos(long walFsyncDelayNanos) {
+        walFsyncDelay = walFsyncDelayNanos;
 
         return this;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b3df5117/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 5136731..cf395a0 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -276,7 +276,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     private FileLockHolder fileLockHolder;
 
     /** Lock wait time. */
-    private final int lockWaitTime;
+    private final long lockWaitTime;
 
     /** */
     private Map<Integer, Map<Integer, T2<Long, WALPointer>>> reservedForExchange;
@@ -2903,10 +2903,10 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         }
 
         /**
-         * @param lockWaitTime During which time thread will try capture file lock.
+         * @param lockWaitTimeMillis During which time thread will try capture file lock.
          * @throws IgniteCheckedException If failed to capture file lock.
          */
-        public void tryLock(int lockWaitTime) throws IgniteCheckedException {
+        public void tryLock(long lockWaitTimeMillis) throws IgniteCheckedException {
             assert lockFile != null;
 
             FileChannel ch = lockFile.getChannel();
@@ -2940,7 +2940,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 String content = null;
 
                 // Try to get lock, if not available wait 1 sec and re-try.
-                for (int i = 0; i < lockWaitTime; i += 1000) {
+                for (int i = 0; i < lockWaitTimeMillis; i += 1000) {
                     try {
                         lock = ch.tryLock(0, 1, false);
                         if (lock != null && lock.isValid()) {
@@ -2964,7 +2964,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 if (content == null)
                     content = readContent();
 
-                failMsg = "Failed to acquire file lock during " + (lockWaitTime / 1000) +
+                failMsg = "Failed to acquire file lock during " + (lockWaitTimeMillis / 1000) +
                     " sec, (locked by " + content + "): " + file.getAbsolutePath();
             }
             catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b3df5117/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
index 5c112fb..897f903 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
@@ -124,7 +124,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     private final int tlbSize;
 
     /** WAL flush frequency. Makes sense only for {@link WALMode#BACKGROUND} log WALMode. */
-    private final int flushFreq;
+    private final long flushFreq;
 
     /** Fsync delay. */
     private final long fsyncDelay;
@@ -229,7 +229,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         mode = psCfg.getWalMode();
         tlbSize = psCfg.getTlbSize();
         flushFreq = psCfg.getWalFlushFrequency();
-        fsyncDelay = psCfg.getWalFsyncDelay();
+        fsyncDelay = psCfg.getWalFsyncDelayNanos();
         alwaysWriteFullPages = psCfg.isAlwaysWriteFullPages();
         ioFactory = psCfg.getFileIOFactory();
         walAutoArchiveAfterInactivity = psCfg.getWalAutoArchiveAfterInactivity();
@@ -1666,7 +1666,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         /** Condition activated each time writeBuffer() completes. Used to wait previously flushed write to complete */
         private final Condition writeComplete = lock.newCondition();
 
-        /** Condition for timed wait of several threads, see {@link PersistentStoreConfiguration#getWalFsyncDelay()} */
+        /** Condition for timed wait of several threads, see {@link PersistentStoreConfiguration#getWalFsyncDelayNanos()} */
         private final Condition fsync = lock.newCondition();
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b3df5117/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index d513071..53d5d3b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -1503,7 +1503,7 @@ public class PlatformConfigurationUtils {
                 .setWalMode(WALMode.fromOrdinal(in.readInt()))
                 .setTlbSize(in.readInt())
                 .setWalFlushFrequency((int) in.readLong())
-                .setWalFsyncDelay(in.readInt())
+                .setWalFsyncDelayNanos(in.readLong())
                 .setWalRecordIteratorBufferSize(in.readInt())
                 .setAlwaysWriteFullPages(in.readBoolean())
                 .setMetricsEnabled(in.readBoolean())
@@ -1535,7 +1535,7 @@ public class PlatformConfigurationUtils {
             w.writeInt(cfg.getWalMode().ordinal());
             w.writeInt(cfg.getTlbSize());
             w.writeLong(cfg.getWalFlushFrequency());
-            w.writeInt(cfg.getWalFsyncDelay());
+            w.writeLong(cfg.getWalFsyncDelayNanos());
             w.writeInt(cfg.getWalRecordIteratorBufferSize());
             w.writeBoolean(cfg.isAlwaysWriteFullPages());
             w.writeBoolean(cfg.isMetricsEnabled());

http://git-wip-us.apache.org/repos/asf/ignite/blob/b3df5117/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java
index 3559845..5b973ed 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java
@@ -39,7 +39,7 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
     private long checkpointingFreq;
 
     /** Lock wait time. */
-    private int lockWaitTime;
+    private long lockWaitTime;
 
     /** */
     private Long checkpointingPageBufSize;
@@ -72,10 +72,10 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
     private int tlbSize;
 
     /** Wal flush frequency. */
-    private int walFlushFreq;
+    private long walFlushFreq;
 
-    /** Wal fsync delay. */
-    private int walFsyncDelay;
+    /** Wal fsync delay in nanoseconds. */
+    private long walFsyncDelay;
 
     /** Wal record iterator buffer size. */
     private int walRecordIterBuffSize;
@@ -114,7 +114,7 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
         walMode = cfg.getWalMode();
         tlbSize = cfg.getTlbSize();
         walFlushFreq = cfg.getWalFlushFrequency();
-        walFsyncDelay = cfg.getWalFsyncDelay();
+        walFsyncDelay = cfg.getWalFsyncDelayNanos();
         walRecordIterBuffSize = cfg.getWalRecordIteratorBufferSize();
         alwaysWriteFullPages = cfg.isAlwaysWriteFullPages();
         subIntervals = cfg.getSubIntervals();
@@ -152,7 +152,7 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
     /**
      * @return Time for wait.
      */
-    public int getLockWaitTime() {
+    public long getLockWaitTime() {
         return lockWaitTime;
     }
 
@@ -229,14 +229,14 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
     /**
      * @return Flush frequency.
      */
-    public int getWalFlushFrequency() {
+    public long getWalFlushFrequency() {
         return walFlushFreq;
     }
 
     /**
      * Gets the fsync delay, in nanoseconds.
      */
-    public int getWalFsyncDelay() {
+    public long getWalFsyncDelay() {
         return walFsyncDelay;
     }
 
@@ -258,7 +258,7 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
     @Override protected void writeExternalData(ObjectOutput out) throws IOException {
         U.writeString(out, persistenceStorePath);
         out.writeLong(checkpointingFreq);
-        out.writeInt(lockWaitTime);
+        out.writeLong(lockWaitTime);
         out.writeObject(checkpointingPageBufSize);
         out.writeInt(checkpointingThreads);
         out.writeInt(walHistSize);
@@ -269,8 +269,8 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
         out.writeBoolean(metricsEnabled);
         U.writeEnum(out, walMode);
         out.writeInt(tlbSize);
-        out.writeInt(walFlushFreq);
-        out.writeInt(walFsyncDelay);
+        out.writeLong(walFlushFreq);
+        out.writeLong(walFsyncDelay);
         out.writeInt(walRecordIterBuffSize);
         out.writeBoolean(alwaysWriteFullPages);
         out.writeInt(subIntervals);
@@ -281,7 +281,7 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
     @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
         persistenceStorePath = U.readString(in);
         checkpointingFreq = in.readLong();
-        lockWaitTime = in.readInt();
+        lockWaitTime = in.readLong();
         checkpointingPageBufSize = (Long)in.readObject();
         checkpointingThreads = in.readInt();
         walHistSize = in.readInt();
@@ -292,8 +292,8 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
         metricsEnabled = in.readBoolean();
         walMode = WALMode.fromOrdinal(in.readByte());
         tlbSize = in.readInt();
-        walFlushFreq = in.readInt();
-        walFsyncDelay = in.readInt();
+        walFlushFreq = in.readLong();
+        walFsyncDelay = in.readLong();
         walRecordIterBuffSize = in.readInt();
         alwaysWriteFullPages = in.readBoolean();
         subIntervals = in.readInt();

http://git-wip-us.apache.org/repos/asf/ignite/blob/b3df5117/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
index 43b17ac..3fc22d6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
@@ -82,7 +82,7 @@ namespace Apache.Ignite.Core.PersistentStore
         /// <summary>
         /// Default value for <see cref="WalFsyncDelayNanos"/>.
         /// </summary>
-        public const int DefaultWalFsyncDelayNanos = 1;
+        public const long DefaultWalFsyncDelayNanos = 1000;
 
         /// <summary>
         /// The default sub intervals.
@@ -137,7 +137,7 @@ namespace Apache.Ignite.Core.PersistentStore
             WalMode = (WalMode)reader.ReadInt();
             TlbSize = reader.ReadInt();
             WalFlushFrequency = reader.ReadLongAsTimespan();
-            WalFsyncDelayNanos = reader.ReadInt();
+            WalFsyncDelayNanos = reader.ReadLong();
             WalRecordIteratorBufferSize = reader.ReadInt();
             AlwaysWriteFullPages = reader.ReadBoolean();
             MetricsEnabled = reader.ReadBoolean();
@@ -166,7 +166,7 @@ namespace Apache.Ignite.Core.PersistentStore
             writer.WriteInt((int)WalMode);
             writer.WriteInt(TlbSize);
             writer.WriteTimeSpanAsLong(WalFlushFrequency);
-            writer.WriteInt(WalFsyncDelayNanos);
+            writer.WriteLong(WalFsyncDelayNanos);
             writer.WriteInt(WalRecordIteratorBufferSize);
             writer.WriteBoolean(AlwaysWriteFullPages);
             writer.WriteBoolean(MetricsEnabled);


[3/8] ignite git commit: IGNITE-5726 Web Console: Fixed duplication of dependencies in maven project.

Posted by ag...@apache.org.
IGNITE-5726 Web Console: Fixed duplication of dependencies in maven project.


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

Branch: refs/heads/ignite-5479
Commit: c5966dd33e12d9e1855c165854b965a8633b1869
Parents: e786beb
Author: Andrey Novikov <an...@gridgain.com>
Authored: Tue Jul 11 17:02:56 2017 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Tue Jul 11 17:02:56 2017 +0700

----------------------------------------------------------------------
 .../app/modules/configuration/generator/Maven.service.js  | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c5966dd3/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js
index 700da4a..81d7d10 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js
@@ -39,7 +39,7 @@ export default class IgniteMavenGenerator {
     }
 
     addDependency(deps, groupId, artifactId, version, jar) {
-        deps.add({groupId, artifactId, version, jar});
+        deps.push({groupId, artifactId, version, jar});
     }
 
     pickDependency(deps, key, dfltVer, igniteVer) {
@@ -78,7 +78,7 @@ export default class IgniteMavenGenerator {
     dependenciesSection(sb, deps) {
         sb.startBlock('<dependencies>');
 
-        deps.forEach((dep) => {
+        _.forEach(deps, (dep) => {
             sb.startBlock('<dependency>');
 
             this.addProperty(sb, 'groupId', dep.groupId);
@@ -154,8 +154,8 @@ export default class IgniteMavenGenerator {
     collectDependencies(cluster, targetVer) {
         const igniteVer = targetVer.ignite;
 
-        const deps = new Set();
-        const storeDeps = new Set();
+        const deps = [];
+        const storeDeps = [];
 
         this.addDependency(deps, 'org.apache.ignite', 'ignite-core', igniteVer);
 
@@ -204,7 +204,7 @@ export default class IgniteMavenGenerator {
         if (cluster.logger && cluster.logger.kind)
             this.pickDependency(deps, cluster.logger.kind, igniteVer);
 
-        return new Set([...deps, ...storeDeps]);
+        return _.uniqWith(deps.concat(...storeDeps), _.isEqual);
     }
 
     /**


[4/8] ignite git commit: IGNITE-5446 Always use late affinity assignment mode - fix .NET tests

Posted by ag...@apache.org.
 IGNITE-5446 Always use late affinity assignment mode - fix .NET tests


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

Branch: refs/heads/ignite-5479
Commit: a9ae95342dd566b210fe835221214ecc67e6ed16
Parents: c5966dd
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Jul 11 14:13:14 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Jul 11 14:13:14 2017 +0300

----------------------------------------------------------------------
 .../Cache/Query/CacheQueriesTest.cs             | 22 ++++++--------------
 1 file changed, 6 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a9ae9534/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
index 4df12a4..62c8230 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
@@ -379,12 +379,13 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             var exp = PopulateCache(cache, loc, cnt, x => x < 50);
 
             // 2. Validate results.
-            var qry = new SqlFieldsQuery("SELECT name, age FROM QueryPerson WHERE age < 50", loc)
+            var qry = new SqlFieldsQuery("SELECT name, age FROM QueryPerson WHERE age < 50")
             {
                 EnableDistributedJoins = distrJoin,
                 EnforceJoinOrder = enforceJoinOrder,
                 Colocated = !distrJoin,
                 ReplicatedOnly = false,
+                Local = loc,
                 Timeout = TimeSpan.FromSeconds(2)
             };
 
@@ -891,29 +892,18 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         {
             var rand = new Random();
 
-            var exp = new HashSet<int>();
-
-            var aff = cache.Ignite.GetAffinity(cache.Name);
-
-            var localNode = cache.Ignite.GetCluster().GetLocalNode();
-
             for (var i = 0; i < cnt; i++)
             {
                 var val = rand.Next(cnt);
 
                 cache.Put(val, new QueryPerson(val.ToString(), val));
-
-                if (expectedEntryFilter(val) && (!loc || aff.IsPrimary(localNode, val)))
-                    exp.Add(val);
             }
 
-            if (loc)
-            {
-                Assert.AreEqual(exp.Count,
-                    cache.GetLocalEntries(CachePeekMode.Primary).Count(x => expectedEntryFilter(x.Key)));
-            }
+            var entries = loc
+                ? cache.GetLocalEntries(CachePeekMode.Primary)
+                : cache;
 
-            return exp;
+            return new HashSet<int>(entries.Select(x => x.Key).Where(expectedEntryFilter));
         }
     }
 


[6/8] ignite git commit: Merge branch 'ignite-2.1.2' into ignite-5479

Posted by ag...@apache.org.
Merge branch 'ignite-2.1.2' into ignite-5479


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

Branch: refs/heads/ignite-5479
Commit: 4775b3d381b3247a095ccd95e21dbda6088df835
Parents: e273e4d 1942db3
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Jul 11 14:27:25 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Jul 11 14:27:25 2017 +0300

----------------------------------------------------------------------
 .../GridCachePartitionExchangeManager.java      |  42 -----
 .../dht/GridClientPartitionTopology.java        |  71 ++++---
 .../dht/GridDhtPartitionTopologyImpl.java       |  27 ++-
 .../dht/preloader/GridDhtPartitionFullMap.java  |  24 +--
 .../GridDhtPartitionsExchangeFuture.java        |   9 +-
 .../persistence/pagemem/PageMemoryImpl.java     |  86 ++++-----
 .../cache/persistence/wal/FileInput.java        |   3 +
 .../utils/PlatformConfigurationUtils.java       |   4 -
 .../internal/processors/query/QueryUtils.java   |   8 +-
 .../CacheLateAffinityAssignmentTest.java        |   2 +-
 ...oreQueryWithMultipleClassesPerCacheTest.java | 185 +++++++++++++++++++
 .../IgnitePdsWithIndexingTestSuite.java         |   2 +
 modules/ml/pom.xml                              |   3 +-
 .../main/java/org/apache/ignite/ml/Model.java   |   1 -
 .../ml/clustering/BaseKMeansClusterer.java      |   2 -
 .../apache/ignite/ml/clustering/Clusterer.java  |   3 +-
 .../clustering/KMeansDistributedClusterer.java  |  46 ++---
 .../ml/clustering/KMeansLocalClusterer.java     |  11 +-
 .../apache/ignite/ml/math/DistanceMeasure.java  |   1 -
 .../ignite/ml/math/EuclideanDistance.java       |   2 +-
 .../java/org/apache/ignite/ml/math/Matrix.java  |   2 -
 .../java/org/apache/ignite/ml/math/Tracer.java  |   1 -
 .../org/apache/ignite/ml/math/VectorUtils.java  |   4 +-
 .../math/exceptions/ConvergenceException.java   |   5 +-
 .../exceptions/MathIllegalStateException.java   |   4 +-
 .../ignite/ml/math/functions/Functions.java     |   8 +-
 .../apache/ignite/ml/math/impls/CacheUtils.java |  23 ++-
 .../ml/math/impls/matrix/AbstractMatrix.java    |   2 -
 .../impls/matrix/SparseDistributedMatrix.java   |   4 +-
 .../impls/storage/matrix/MapWrapperStorage.java |   7 +-
 .../matrix/SparseDistributedMatrixStorage.java  |   7 +-
 .../vector/SparseLocalOnHeapVectorStorage.java  |   8 +-
 .../ml/math/impls/vector/MapWrapperVector.java  |   3 +-
 .../apache/ignite/ml/math/util/MatrixUtil.java  |   1 -
 .../ignite/ml/math/d3-matrix-template.html      |   2 +
 .../ignite/ml/math/d3-vector-template.html      |   2 +
 .../org/apache/ignite/ml/IgniteMLTestSuite.java |   4 +-
 .../ml/clustering/ClusteringTesetSuite.java     |  32 ----
 .../ml/clustering/ClusteringTestSuite.java      |  32 ++++
 .../KMeansDistributedClustererTest.java         |  28 ++-
 .../ignite/ml/math/MathImplLocalTestSuite.java  |   1 -
 .../org/apache/ignite/ml/math/TracerTest.java   |   3 +
 .../matrix/SparseDistributedMatrixTest.java     |  24 ++-
 .../OLSMultipleLinearRegressionTest.java        |   2 +-
 .../Apache.Ignite.Core.Tests.csproj             |   1 +
 .../Cache/Query/CacheQueriesTest.cs             |  18 +-
 .../Config/cache-query.xml                      |   1 -
 .../Deployment/CacheGetFunc.cs                  |  50 +++++
 .../Deployment/PeerAssemblyLoadingTest.cs       |  24 +++
 .../IgniteConfigurationSerializerTest.cs        |   2 -
 .../IgniteConfigurationTest.cs                  |   3 +-
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  13 +-
 .../org/apache/ignite/spark/IgniteRDD.scala     |   6 +-
 .../apache/ignite/spark/impl/IgniteSqlRDD.scala |   6 +-
 .../spark/JavaEmbeddedIgniteRDDSelfTest.java    |   5 -
 modules/web-console/frontend/app/app.js         |   2 +
 .../cluster-select/cluster-select.controller.js |  72 ++++----
 .../cluster-select/cluster-select.pug           |  10 +-
 .../frontend/app/filters/id8.filter.js          |  20 ++
 .../app/modules/agent/AgentManager.service.js   |   4 +-
 .../configuration/generator/Maven.service.js    |  10 +-
 .../views/templates/agent-download.tpl.pug      |   4 +-
 62 files changed, 617 insertions(+), 375 deletions(-)
----------------------------------------------------------------------


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


[5/8] ignite git commit: .NET: Fix code analysis warnings for deprecated IsLateAffinityAssignment

Posted by ag...@apache.org.
.NET: Fix code analysis warnings for deprecated IsLateAffinityAssignment


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

Branch: refs/heads/ignite-5479
Commit: 1942db3af6538e6d1f0731b52eda94071f533ccb
Parents: a9ae953
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Jul 11 14:23:56 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Jul 11 14:23:56 2017 +0300

----------------------------------------------------------------------
 modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1942db3a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
index 86155a6..4d04348 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
@@ -970,6 +970,8 @@ namespace Apache.Ignite.Core
         /// <para />
         /// If not provided, default value is <see cref="DefaultIsLateAffinityAssignment"/>.
         /// </summary>
+        [SuppressMessage("Microsoft.Performance", "CA1822:MarkMembersAsStatic")]
+        [SuppressMessage("Microsoft.Usage", "CA1801:ReviewUnusedParameters", MessageId = "value")]
         [DefaultValue(DefaultIsLateAffinityAssignment)]
         [Obsolete("No longer supported, always true.")]
         public bool IsLateAffinityAssignment


[8/8] ignite git commit: IGNITE-5479 - Clean up public API

Posted by ag...@apache.org.
IGNITE-5479 - Clean up public API


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

Branch: refs/heads/ignite-5479
Commit: 727df1871d40db6f1c5c4faac33488255fccb0d1
Parents: 8a64081
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Jul 11 15:26:11 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Jul 11 15:26:11 2017 +0300

----------------------------------------------------------------------
 .../ignite/configuration/MemoryConfiguration.java       |  2 +-
 .../ignite/configuration/MemoryPolicyConfiguration.java |  2 +-
 .../persistence/IgniteCacheDatabaseSharedManager.java   | 12 ++++++++++--
 3 files changed, 12 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/727df187/modules/core/src/main/java/org/apache/ignite/configuration/MemoryConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryConfiguration.java
index 5cf6cb7..97392de 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryConfiguration.java
@@ -64,7 +64,7 @@ public class MemoryConfiguration implements Serializable {
 
     /** Default memory policy start size (256 MB). */
     @SuppressWarnings("UnnecessaryBoxing")
-    public static final Long DFLT_MEMORY_POLICY_INITIAL_SIZE = new Long(256L * 1024 * 1024);
+    public static final long DFLT_MEMORY_POLICY_INITIAL_SIZE = 256L * 1024 * 1024;
 
     /** Fraction of available memory to allocate for default MemoryPolicy. */
     private static final double DFLT_MEMORY_POLICY_FRACTION = 0.8;

http://git-wip-us.apache.org/repos/asf/ignite/blob/727df187/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
index b496901..fec20d3 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
@@ -78,7 +78,7 @@ public final class MemoryPolicyConfiguration implements Serializable {
     private String name = DFLT_MEM_PLC_DEFAULT_NAME;
 
     /** Memory policy start size. */
-    private Long initialSize = MemoryConfiguration.DFLT_MEMORY_POLICY_INITIAL_SIZE;
+    private long initialSize;
 
     /** Memory policy maximum size. */
     private long maxSize = MemoryConfiguration.DFLT_MEMORY_POLICY_MAX_SIZE;

http://git-wip-us.apache.org/repos/asf/ignite/blob/727df187/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
index c5f174c..8d03689 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
@@ -454,6 +454,14 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      * @throws IgniteCheckedException If config is invalid.
      */
     private void checkPolicySize(MemoryPolicyConfiguration plcCfg) throws IgniteCheckedException {
+        boolean dfltInitSize = false;
+
+        if (plcCfg.getInitialSize() == 0) {
+            plcCfg.setInitialSize(DFLT_MEMORY_POLICY_INITIAL_SIZE);
+
+            dfltInitSize = true;
+        }
+
         if (plcCfg.getInitialSize() < MIN_PAGE_MEMORY_SIZE)
             throw new IgniteCheckedException("MemoryPolicy must have size more than 10MB (use " +
                 "MemoryPolicyConfiguration.initialSize property to set correct size in bytes) " +
@@ -461,8 +469,8 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
             );
 
         if (plcCfg.getMaxSize() < plcCfg.getInitialSize()) {
-            // We will know for sure if initialSize has been changed if we compare Longs by "==".
-            if (plcCfg.getInitialSize() == DFLT_MEMORY_POLICY_INITIAL_SIZE) {
+            // If initial size was not set, use the max size.
+            if (dfltInitSize) {
                 plcCfg.setInitialSize(plcCfg.getMaxSize());
 
                 LT.warn(log, "MemoryPolicy maxSize=" + U.readableSize(plcCfg.getMaxSize(), true) +


[7/8] ignite git commit: IGNITE-5479 - Clean up public API

Posted by ag...@apache.org.
IGNITE-5479 - Clean up public API


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

Branch: refs/heads/ignite-5479
Commit: 8a64081c6161c87d0a9c392308b06bb890c54d27
Parents: 4775b3d
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Jul 11 15:11:52 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Jul 11 15:12:15 2017 +0300

----------------------------------------------------------------------
 .../configuration/PersistentStoreConfiguration.java       | 10 +++++-----
 .../cache/persistence/GridCacheDatabaseSharedManager.java |  4 ++--
 .../platform/utils/PlatformConfigurationUtils.java        |  4 +---
 .../visor/node/VisorPersistentStoreConfiguration.java     |  6 +++---
 4 files changed, 11 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8a64081c/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
index aa34bba..7706bf3 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
@@ -93,7 +93,7 @@ public class PersistentStoreConfiguration implements Serializable {
     private long lockWaitTime = DFLT_LOCK_WAIT_TIME;
 
     /** */
-    private Long checkpointingPageBufSize;
+    private long checkpointingPageBufSize;
 
     /** */
     private int checkpointingThreads = DFLT_CHECKPOINTING_THREADS;
@@ -199,10 +199,10 @@ public class PersistentStoreConfiguration implements Serializable {
     /**
      * Gets amount of memory allocated for a checkpointing temporary buffer.
      *
-     * @return Checkpointing page buffer size in bytes or {@code null} for Ignite
+     * @return Checkpointing page buffer size in bytes or {@code 0} for Ignite
      *      to choose the buffer size automatically.
      */
-    public Long getCheckpointingPageBufferSize() {
+    public long getCheckpointingPageBufferSize() {
         return checkpointingPageBufSize;
     }
 
@@ -211,11 +211,11 @@ public class PersistentStoreConfiguration implements Serializable {
      * copies of pages that are being written to disk and being update in parallel while the checkpointing is in
      * progress.
      *
-     * @param checkpointingPageBufSize Checkpointing page buffer size in bytes or {@code null} for Ignite to
+     * @param checkpointingPageBufSize Checkpointing page buffer size in bytes or {@code 0} for Ignite to
      *      choose the buffer size automatically.
      * @return {@code this} for chaining.
      */
-    public PersistentStoreConfiguration setCheckpointingPageBufferSize(Long checkpointingPageBufSize) {
+    public PersistentStoreConfiguration setCheckpointingPageBufferSize(long checkpointingPageBufSize) {
         this.checkpointingPageBufSize = checkpointingPageBufSize;
 
         return this;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a64081c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 08b55bb..122d460 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -377,7 +377,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
      *
      */
     private void initDataBase() {
-        Long cpBufSize = persistenceCfg.getCheckpointingPageBufferSize();
+        long cpBufSize = persistenceCfg.getCheckpointingPageBufferSize();
 
         if (persistenceCfg.getCheckpointingThreads() > 1)
             asyncRunner = new ThreadPoolExecutor(
@@ -390,7 +390,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
         // Intentionally use identity comparison to check if configuration default has changed.
         //noinspection NumberEquality
-        if (cpBufSize == null) {
+        if (cpBufSize == 0L) {
             cpBufSize = PersistentStoreConfiguration.DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE;
 
             MemoryConfiguration memCfg = cctx.kernalContext().config().getMemoryConfiguration();

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a64081c/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index a909434..8c9f5e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -1520,9 +1520,7 @@ public class PlatformConfigurationUtils {
 
             w.writeString(cfg.getPersistentStorePath());
             w.writeLong(cfg.getCheckpointingFrequency());
-            w.writeLong(cfg.getCheckpointingPageBufferSize() == null ?
-                PersistentStoreConfiguration.DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE :
-                cfg.getCheckpointingPageBufferSize());
+            w.writeLong(cfg.getCheckpointingPageBufferSize());
             w.writeInt(cfg.getCheckpointingThreads());
             w.writeLong(cfg.getLockWaitTime());
             w.writeInt(cfg.getWalHistorySize());

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a64081c/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java
index 5b973ed..abbab4c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java
@@ -42,7 +42,7 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
     private long lockWaitTime;
 
     /** */
-    private Long checkpointingPageBufSize;
+    private long checkpointingPageBufSize;
 
     /** */
     private int checkpointingThreads;
@@ -259,7 +259,7 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
         U.writeString(out, persistenceStorePath);
         out.writeLong(checkpointingFreq);
         out.writeLong(lockWaitTime);
-        out.writeObject(checkpointingPageBufSize);
+        out.writeLong(checkpointingPageBufSize);
         out.writeInt(checkpointingThreads);
         out.writeInt(walHistSize);
         out.writeInt(walSegments);
@@ -282,7 +282,7 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
         persistenceStorePath = U.readString(in);
         checkpointingFreq = in.readLong();
         lockWaitTime = in.readLong();
-        checkpointingPageBufSize = (Long)in.readObject();
+        checkpointingPageBufSize = in.readLong();
         checkpointingThreads = in.readInt();
         walHistSize = in.readInt();
         walSegments = in.readInt();


[2/8] ignite git commit: IGNITE-5479 - Changed default checkpoint buffer size

Posted by ag...@apache.org.
IGNITE-5479 - Changed default checkpoint buffer size


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

Branch: refs/heads/ignite-5479
Commit: e273e4d4aa28b339694fbfd0951ed5ff1b232cfb
Parents: b3df511
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Jul 10 20:01:39 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Jul 10 20:01:39 2017 +0300

----------------------------------------------------------------------
 .../configuration/PersistentStoreConfiguration.java  | 15 ++++++++-------
 .../persistence/GridCacheDatabaseSharedManager.java  |  8 +++++---
 .../platform/utils/PlatformConfigurationUtils.java   |  4 +++-
 3 files changed, 16 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e273e4d4/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
index 1d4e2ec..aa34bba 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
@@ -44,9 +44,8 @@ public class PersistentStoreConfiguration implements Serializable {
     /** Default length of interval over which rate-based metric is calculated. */
     public static final int DFLT_RATE_TIME_INTERVAL_MILLIS = 60_000;
 
-    /** */
-    @SuppressWarnings("UnnecessaryBoxing")
-    public static final Long DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE = new Long(256L * 1024 * 1024);
+    /** Default checkpointing page buffer size (may be adjusted by Ignite). */
+    public static final Long DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE = 256L * 1024 * 1024 * 1024;
 
     /** Default number of checkpointing threads. */
     public static final int DFLT_CHECKPOINTING_THREADS = 1;
@@ -94,7 +93,7 @@ public class PersistentStoreConfiguration implements Serializable {
     private long lockWaitTime = DFLT_LOCK_WAIT_TIME;
 
     /** */
-    private Long checkpointingPageBufSize = DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE;
+    private Long checkpointingPageBufSize;
 
     /** */
     private int checkpointingThreads = DFLT_CHECKPOINTING_THREADS;
@@ -200,7 +199,8 @@ public class PersistentStoreConfiguration implements Serializable {
     /**
      * Gets amount of memory allocated for a checkpointing temporary buffer.
      *
-     * @return checkpointing page buffer size in bytes.
+     * @return Checkpointing page buffer size in bytes or {@code null} for Ignite
+     *      to choose the buffer size automatically.
      */
     public Long getCheckpointingPageBufferSize() {
         return checkpointingPageBufSize;
@@ -211,10 +211,11 @@ public class PersistentStoreConfiguration implements Serializable {
      * copies of pages that are being written to disk and being update in parallel while the checkpointing is in
      * progress.
      *
-     * @param checkpointingPageBufSize checkpointing page buffer size in bytes.
+     * @param checkpointingPageBufSize Checkpointing page buffer size in bytes or {@code null} for Ignite to
+     *      choose the buffer size automatically.
      * @return {@code this} for chaining.
      */
-    public PersistentStoreConfiguration setCheckpointingPageBufferSize(long checkpointingPageBufSize) {
+    public PersistentStoreConfiguration setCheckpointingPageBufferSize(Long checkpointingPageBufSize) {
         this.checkpointingPageBufSize = checkpointingPageBufSize;
 
         return this;

http://git-wip-us.apache.org/repos/asf/ignite/blob/e273e4d4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index cf395a0..08b55bb 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -374,9 +374,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     }
 
     /**
-     * @throws IgniteCheckedException If failed.
+     *
      */
-    private void initDataBase() throws IgniteCheckedException {
+    private void initDataBase() {
         Long cpBufSize = persistenceCfg.getCheckpointingPageBufferSize();
 
         if (persistenceCfg.getCheckpointingThreads() > 1)
@@ -390,7 +390,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
         // Intentionally use identity comparison to check if configuration default has changed.
         //noinspection NumberEquality
-        if (cpBufSize == PersistentStoreConfiguration.DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE) {
+        if (cpBufSize == null) {
+            cpBufSize = PersistentStoreConfiguration.DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE;
+
             MemoryConfiguration memCfg = cctx.kernalContext().config().getMemoryConfiguration();
 
             assert memCfg != null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/e273e4d4/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index 53d5d3b..fee9856 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -1524,7 +1524,9 @@ public class PlatformConfigurationUtils {
 
             w.writeString(cfg.getPersistentStorePath());
             w.writeLong(cfg.getCheckpointingFrequency());
-            w.writeLong(cfg.getCheckpointingPageBufferSize());
+            w.writeLong(cfg.getCheckpointingPageBufferSize() == null ?
+                PersistentStoreConfiguration.DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE :
+                cfg.getCheckpointingPageBufferSize());
             w.writeInt(cfg.getCheckpointingThreads());
             w.writeLong(cfg.getLockWaitTime());
             w.writeInt(cfg.getWalHistorySize());