This is an automated email from the ASF dual-hosted git repository. dgovorukhin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/ignite.git
The following commit(s) were added to refs/heads/master by this push: new a736544 IGNITE-11993 Print warning if awaiting next wal segment it too long - Fixes #6703. a736544 is described below commit a7365445e53d24aa80a54127b5407cdbd30c20f8 Author: Kirill Tkalenko <tkalkir...@yandex.ru> AuthorDate: Tue Jul 23 13:27:34 2019 +0300 IGNITE-11993 Print warning if awaiting next wal segment it too long - Fixes #6703. Signed-off-by: Dmitriy Govorukhin <dmitriy.govoruk...@gmail.com> --- .../org/apache/ignite/IgniteSystemProperties.java | 7 +++++++ .../persistence/wal/FileWriteAheadLogManager.java | 22 ++++++++++++++++++++++ 2 files changed, 29 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index 66acae0..9d785d1 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -914,6 +914,13 @@ public final class IgniteSystemProperties { public static final String IGNITE_THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE = "IGNITE_THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE"; /** + * Threshold time (in millis) to print warning to log if waiting for next wal segment took longer than the threshold. + * + * Default value is 1000 ms. + */ + public static final String IGNITE_THRESHOLD_WAIT_TIME_NEXT_WAL_SEGMENT = "IGNITE_THRESHOLD_WAIT_TIME_NEXT_WAL_SEGMENT"; + + /** * Count of WAL compressor worker threads. Default value is 4. */ public static final String IGNITE_WAL_COMPRESSOR_WORKER_THREAD_CNT = "IGNITE_WAL_COMPRESSOR_WORKER_THREAD_CNT"; 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 9f2c352..d337944 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 @@ -135,6 +135,7 @@ import static java.nio.file.StandardOpenOption.CREATE; import static java.nio.file.StandardOpenOption.READ; import static java.nio.file.StandardOpenOption.WRITE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_CHECKPOINT_TRIGGER_ARCHIVE_SIZE_PERCENTAGE; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_THRESHOLD_WAIT_TIME_NEXT_WAL_SEGMENT; import static org.apache.ignite.IgniteSystemProperties.IGNITE_THRESHOLD_WAL_ARCHIVE_SIZE_PERCENTAGE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_COMPRESSOR_WORKER_THREAD_CNT; import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_MMAP; @@ -234,6 +235,12 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl private final int WAL_COMPRESSOR_WORKER_THREAD_CNT = IgniteSystemProperties.getInteger(IGNITE_WAL_COMPRESSOR_WORKER_THREAD_CNT, 4); + /** + * Threshold time to print warning to log if awaiting for next wal segment took too long (exceeded this threshold). + */ + private static final long THRESHOLD_WAIT_TIME_NEXT_WAL_SEGMENT = + IgniteSystemProperties.getLong(IGNITE_THRESHOLD_WAIT_TIME_NEXT_WAL_SEGMENT, 1000L); + /** */ private final boolean alwaysWriteFullPages; @@ -1565,9 +1572,24 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl return new File(walWorkDir, FileDescriptor.fileName(curIdx + 1)); } + long absNextIdxStartTime = System.nanoTime(); + // Signal to archiver that we are done with the segment and it can be archived. long absNextIdx = archiver0.nextAbsoluteSegmentIndex(); + long absNextIdxWaitTime = U.nanosToMillis(System.nanoTime() - absNextIdxStartTime); + + if (absNextIdxWaitTime > THRESHOLD_WAIT_TIME_NEXT_WAL_SEGMENT) { + log.warning( + String.format("Waiting for next wal segment was too long " + + "[waitingTime=%s, curIdx=%s, absNextIdx=%s, walSegments=%s]", + absNextIdxWaitTime, + curIdx, + absNextIdx, + dsCfg.getWalSegments()) + ); + } + long segmentIdx = absNextIdx % dsCfg.getWalSegments(); return new File(walWorkDir, FileDescriptor.fileName(segmentIdx));