rpuch commented on code in PR #7250:
URL: https://github.com/apache/ignite-3/pull/7250#discussion_r2651080860


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointReadWriteLockMetrics.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.checkpoint;
+
+import org.apache.ignite.internal.metrics.DistributionMetric;
+import org.apache.ignite.internal.metrics.LongAdderMetric;
+
+/**
+ * Metrics for checkpoint read/write lock operations.
+ *
+ * <p>This metric source tracks performance and contention characteristics of 
checkpoint read locks
+ * acquired by normal operations during database operation. These locks 
coordinate with the checkpoint
+ * write lock to ensure consistency.

Review Comment:
   The last sentence looks like it's generated by AI. It doesn't add any value, 
just takes space. Should we keep it?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointReadWriteLockMetrics.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.checkpoint;
+
+import org.apache.ignite.internal.metrics.DistributionMetric;
+import org.apache.ignite.internal.metrics.LongAdderMetric;
+
+/**
+ * Metrics for checkpoint read/write lock operations.
+ *
+ * <p>This metric source tracks performance and contention characteristics of 
checkpoint read locks
+ * acquired by normal operations during database operation. These locks 
coordinate with the checkpoint
+ * write lock to ensure consistency.
+ */
+public class CheckpointReadWriteLockMetrics {
+    private static final long[] LOCK_ACQUISITION_NANOS = {

Review Comment:
   Those are bounds



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointReadWriteLockMetrics.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.checkpoint;
+
+import org.apache.ignite.internal.metrics.DistributionMetric;
+import org.apache.ignite.internal.metrics.LongAdderMetric;
+
+/**
+ * Metrics for checkpoint read/write lock operations.
+ *
+ * <p>This metric source tracks performance and contention characteristics of 
checkpoint read locks

Review Comment:
   ```suggestion
    * <p>This metric source tracks performance and contention characteristics 
of checkpoint read lock
   ```



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointReadWriteLock.java:
##########
@@ -181,15 +192,37 @@ public boolean hasQueuedWriters() {
         return checkpointLock.hasQueuedWriters();
     }
 
-    private void onReadLock(long start, boolean taken) {
-        long elapsed = coarseCurrentTimeMillis() - start;
+    private void onReadLock(long startNanos, boolean taken) {
+        metrics.decrementReadLockWaitingThreads();
+
+        long currentNanos = System.nanoTime();
+        long elapsedNanos = TimeUnit.NANOSECONDS.toMillis(currentNanos - 
startNanos);
 
         if (taken) {
-            checkpointReadLockHoldCount.set(checkpointReadLockHoldCount.get() 
+ 1);
+            int newLockCount = checkpointReadLockHoldCount.get() + 1;
+            checkpointReadLockHoldCount.set(newLockCount);
+
+            // We only record acquisition time on first lock acquisition (not 
on reentrant locks).

Review Comment:
   ```suggestion
               // We only record acquisition time on first lock acquisition 
(not on reentry).
   ```



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointReadWriteLockMetrics.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.checkpoint;
+
+import org.apache.ignite.internal.metrics.DistributionMetric;
+import org.apache.ignite.internal.metrics.LongAdderMetric;
+
+/**
+ * Metrics for checkpoint read/write lock operations.
+ *
+ * <p>This metric source tracks performance and contention characteristics of 
checkpoint read locks
+ * acquired by normal operations during database operation. These locks 
coordinate with the checkpoint
+ * write lock to ensure consistency.
+ */
+public class CheckpointReadWriteLockMetrics {
+    private static final long[] LOCK_ACQUISITION_NANOS = {
+            1_000,           // 1µs   - uncontended, fast path
+            10_000,          // 10µs  - minor contention
+            100_000,         // 100µs - moderate contention
+            1_000_000,       // 1ms   - high contention
+            10_000_000,      // 10ms  - checkpoint in progress?
+            100_000_000,     // 100ms - severe contention, reported as warning 
in logs
+            1_000_000_000    // 1s    - pathological case, shall be treated as 
an emergency error

Review Comment:
   What is the source of the comments on each of the durations?



##########
modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointTimeoutLockTest.java:
##########
@@ -407,4 +417,46 @@ private Checkpointer newCheckpointer(Thread runner, 
CompletableFuture<?> future)
 
         return checkpointer;
     }
+
+    @Test
+    void testCheckpointReadLockMetrics() throws Exception {
+        CheckpointMetricSource metricSource = new 
CheckpointMetricSource("test");
+        CheckpointReadWriteLockMetrics metrics = new 
CheckpointReadWriteLockMetrics(metricSource);
+        CheckpointReadWriteLock readWriteLock = newReadWriteLock(metrics);
+
+        timeoutLock = new CheckpointTimeoutLock(
+                readWriteLock,
+                10_000,
+                () -> NOT_REQUIRED,
+                mock(Checkpointer.class),
+                mock(FailureManager.class)
+        );
+
+        timeoutLock.start();
+
+        try {
+            // Verify metrics start at zero
+            
assertTrue(Arrays.stream(metrics.readLockAcquisitionTime().value()).allMatch(it 
-> it == 0L));

Review Comment:
   Is there a Hamcrest matcher that makes sure that every element satisfies a 
condition? Such an assertion would give more information on failure.
   
   I would suggest to do the same with the assertions that follow



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointReadWriteLockMetrics.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.checkpoint;
+
+import org.apache.ignite.internal.metrics.DistributionMetric;
+import org.apache.ignite.internal.metrics.LongAdderMetric;
+
+/**
+ * Metrics for checkpoint read/write lock operations.
+ *
+ * <p>This metric source tracks performance and contention characteristics of 
checkpoint read locks
+ * acquired by normal operations during database operation. These locks 
coordinate with the checkpoint

Review Comment:
   ```suggestion
    * acquired by normal operations during database operation. This lock 
coordinate with the checkpoint
   ```



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointManager.java:
##########
@@ -124,9 +124,12 @@ public CheckpointManager(
                 ? new 
ReentrantReadWriteLockWithTracking(Loggers.forClass(CheckpointReadWriteLock.class),
 logReadLockThresholdTimeout)
                 : new ReentrantReadWriteLockWithTracking();
 
+        CheckpointReadWriteLockMetrics readWriteLockMetrics = new 
CheckpointReadWriteLockMetrics(checkpointMetricSource);

Review Comment:
   ```suggestion
           var readWriteLockMetrics = new 
CheckpointReadWriteLockMetrics(checkpointMetricSource);
   ```



##########
modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointTimeoutLockTest.java:
##########
@@ -407,4 +417,46 @@ private Checkpointer newCheckpointer(Thread runner, 
CompletableFuture<?> future)
 
         return checkpointer;
     }
+
+    @Test
+    void testCheckpointReadLockMetrics() throws Exception {
+        CheckpointMetricSource metricSource = new 
CheckpointMetricSource("test");
+        CheckpointReadWriteLockMetrics metrics = new 
CheckpointReadWriteLockMetrics(metricSource);
+        CheckpointReadWriteLock readWriteLock = newReadWriteLock(metrics);
+
+        timeoutLock = new CheckpointTimeoutLock(
+                readWriteLock,
+                10_000,
+                () -> NOT_REQUIRED,
+                mock(Checkpointer.class),
+                mock(FailureManager.class)
+        );
+
+        timeoutLock.start();
+
+        try {
+            // Verify metrics start at zero
+            
assertTrue(Arrays.stream(metrics.readLockAcquisitionTime().value()).allMatch(it 
-> it == 0L));
+
+            // Acquire and immediately release the lock
+            timeoutLock.checkpointReadLock();
+            timeoutLock.checkpointReadUnlock();
+
+            // Verify acquisition was recorded
+            
assertTrue(Arrays.stream(metrics.readLockAcquisitionTime().value()).anyMatch(it 
-> it == 1L));
+
+            // Verify hold time distribution was recorded
+            
assertTrue(Arrays.stream(metrics.readLockHoldTime().value()).anyMatch(it -> it 
== 1L));
+
+            readWriteLock.writeLock();
+            runAsync(() -> {
+                timeoutLock.checkpointReadLock();
+                timeoutLock.checkpointReadUnlock();
+            });
+            await().untilAsserted(() -> 
assertThat(metrics.readLockWaitingThreads().value(), is(1L)));
+            readWriteLock.writeUnlock();

Review Comment:
   Let's assert that the thread counter drops to zero



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointReadWriteLock.java:
##########
@@ -181,15 +192,37 @@ public boolean hasQueuedWriters() {
         return checkpointLock.hasQueuedWriters();
     }
 
-    private void onReadLock(long start, boolean taken) {
-        long elapsed = coarseCurrentTimeMillis() - start;
+    private void onReadLock(long startNanos, boolean taken) {
+        metrics.decrementReadLockWaitingThreads();
+
+        long currentNanos = System.nanoTime();
+        long elapsedNanos = TimeUnit.NANOSECONDS.toMillis(currentNanos - 
startNanos);

Review Comment:
   ```suggestion
           long elapsedNanos = currentNanos - startNanos;
   ```



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointReadWriteLockMetrics.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.checkpoint;
+
+import org.apache.ignite.internal.metrics.DistributionMetric;
+import org.apache.ignite.internal.metrics.LongAdderMetric;
+
+/**
+ * Metrics for checkpoint read/write lock operations.
+ *
+ * <p>This metric source tracks performance and contention characteristics of 
checkpoint read locks
+ * acquired by normal operations during database operation. These locks 
coordinate with the checkpoint
+ * write lock to ensure consistency.
+ */
+public class CheckpointReadWriteLockMetrics {
+    private static final long[] LOCK_ACQUISITION_NANOS = {
+            1_000,           // 1µs   - uncontended, fast path
+            10_000,          // 10µs  - minor contention
+            100_000,         // 100µs - moderate contention
+            1_000_000,       // 1ms   - high contention
+            10_000_000,      // 10ms  - checkpoint in progress?
+            100_000_000,     // 100ms - severe contention, reported as warning 
in logs
+            1_000_000_000    // 1s    - pathological case, shall be treated as 
an emergency error
+    };
+
+    private static final long[] LOCK_HOLD_NANOS = {
+            1_000,           // 1µs    - very fast operation (single field 
update)
+            10_000,          // 10µs   - fast single-page operation
+            100_000,         // 100µs  - multi-page operation
+            1_000_000,       // 1ms    - complex operation
+            10_000_000,      // 10ms   - batch operation
+            100_000_000,     // 100ms  - large batch or slow I/O
+            1_000_000_000    // 1s     - pathologically large operation

Review Comment:
   ```suggestion
               1_000_000_000    // 1s     - pathologically long operation
   ```



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointReadWriteLock.java:
##########
@@ -181,15 +192,37 @@ public boolean hasQueuedWriters() {
         return checkpointLock.hasQueuedWriters();
     }
 
-    private void onReadLock(long start, boolean taken) {
-        long elapsed = coarseCurrentTimeMillis() - start;
+    private void onReadLock(long startNanos, boolean taken) {
+        metrics.decrementReadLockWaitingThreads();
+
+        long currentNanos = System.nanoTime();
+        long elapsedNanos = TimeUnit.NANOSECONDS.toMillis(currentNanos - 
startNanos);
 
         if (taken) {
-            checkpointReadLockHoldCount.set(checkpointReadLockHoldCount.get() 
+ 1);
+            int newLockCount = checkpointReadLockHoldCount.get() + 1;
+            checkpointReadLockHoldCount.set(newLockCount);
+
+            // We only record acquisition time on first lock acquisition (not 
on reentrant locks).
+            if (newLockCount == 1) {
+                checkpointReadLockAcquiredTime.set(currentNanos);
+            }
+            metrics.recordReadLockAcquisitionTime(elapsedNanos);
         }
 
-        if (elapsed > LONG_LOCK_THRESHOLD_MILLIS) {
-            log.warn(LONG_LOCK_THROTTLE_KEY, "Checkpoint read lock took {} ms 
to acquire.", elapsed);
+        if (elapsedNanos > LONG_LOCK_THRESHOLD_MILLIS) {
+            log.warn(LONG_LOCK_THROTTLE_KEY, "Checkpoint read lock took {} ms 
to acquire.", elapsedNanos);

Review Comment:
   The parameter is in nanos, but the log message claims it's in millis



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointReadWriteLock.java:
##########
@@ -181,15 +192,37 @@ public boolean hasQueuedWriters() {
         return checkpointLock.hasQueuedWriters();
     }
 
-    private void onReadLock(long start, boolean taken) {
-        long elapsed = coarseCurrentTimeMillis() - start;
+    private void onReadLock(long startNanos, boolean taken) {
+        metrics.decrementReadLockWaitingThreads();
+
+        long currentNanos = System.nanoTime();
+        long elapsedNanos = TimeUnit.NANOSECONDS.toMillis(currentNanos - 
startNanos);
 
         if (taken) {
-            checkpointReadLockHoldCount.set(checkpointReadLockHoldCount.get() 
+ 1);
+            int newLockCount = checkpointReadLockHoldCount.get() + 1;
+            checkpointReadLockHoldCount.set(newLockCount);
+
+            // We only record acquisition time on first lock acquisition (not 
on reentrant locks).
+            if (newLockCount == 1) {
+                checkpointReadLockAcquiredTime.set(currentNanos);
+            }
+            metrics.recordReadLockAcquisitionTime(elapsedNanos);
         }
 
-        if (elapsed > LONG_LOCK_THRESHOLD_MILLIS) {
-            log.warn(LONG_LOCK_THROTTLE_KEY, "Checkpoint read lock took {} ms 
to acquire.", elapsed);
+        if (elapsedNanos > LONG_LOCK_THRESHOLD_MILLIS) {
+            log.warn(LONG_LOCK_THROTTLE_KEY, "Checkpoint read lock took {} ms 
to acquire.", elapsedNanos);
+        }
+    }
+
+    private void onReadUnlock() {
+        int newLockCount = checkpointReadLockHoldCount.get() - 1;
+        checkpointReadLockHoldCount.set(newLockCount);
+        if (newLockCount == 0) {
+            // Fully unlocked - record hold duration.
+            Long acquiredTimeNanos = checkpointReadLockAcquiredTime.get();
+            checkpointReadLockAcquiredTime.set(null);

Review Comment:
   Why do we need this? Will it be rewritten on next acquisition anyway?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointReadWriteLockMetrics.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.checkpoint;
+
+import org.apache.ignite.internal.metrics.DistributionMetric;
+import org.apache.ignite.internal.metrics.LongAdderMetric;
+
+/**
+ * Metrics for checkpoint read/write lock operations.
+ *
+ * <p>This metric source tracks performance and contention characteristics of 
checkpoint read locks
+ * acquired by normal operations during database operation. These locks 
coordinate with the checkpoint
+ * write lock to ensure consistency.
+ */
+public class CheckpointReadWriteLockMetrics {
+    private static final long[] LOCK_ACQUISITION_NANOS = {
+            1_000,           // 1µs   - uncontended, fast path
+            10_000,          // 10µs  - minor contention
+            100_000,         // 100µs - moderate contention
+            1_000_000,       // 1ms   - high contention
+            10_000_000,      // 10ms  - checkpoint in progress?
+            100_000_000,     // 100ms - severe contention, reported as warning 
in logs
+            1_000_000_000    // 1s    - pathological case, shall be treated as 
an emergency error
+    };
+
+    private static final long[] LOCK_HOLD_NANOS = {

Review Comment:
   Bounds



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to