Skip to content

Commit a625684

Browse files
authored
IGNITE-27125 Add extra debug logging and assertions. (#7278)
1 parent 561e01e commit a625684

File tree

2 files changed

+17
-3
lines changed
  • modules

2 files changed

+17
-3
lines changed

modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/server/WatchProcessor.java

Lines changed: 15 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -36,6 +36,7 @@
3636
import java.util.Arrays;
3737
import java.util.List;
3838
import java.util.OptionalLong;
39+
import java.util.Set;
3940
import java.util.concurrent.CompletableFuture;
4041
import java.util.concurrent.CopyOnWriteArrayList;
4142
import java.util.concurrent.ExecutorService;
@@ -47,6 +48,7 @@
4748
import java.util.concurrent.atomic.AtomicBoolean;
4849
import java.util.function.Consumer;
4950
import java.util.function.Supplier;
51+
import java.util.stream.Collectors;
5052
import org.apache.ignite.internal.close.ManuallyCloseable;
5153
import org.apache.ignite.internal.failure.FailureContext;
5254
import org.apache.ignite.internal.failure.FailureProcessor;
@@ -115,6 +117,10 @@ public interface EntryReader {
115117

116118
private volatile WatchEventHandlingCallback watchEventHandlingCallback;
117119

120+
// This field is used in assertions only. It was added in order to ease the debug of a tricky problem that is nearly impossible to
121+
// reproduce.
122+
private volatile long revision = -1;
123+
118124
/** Executor for processing watch events. */
119125
private final ExecutorService watchExecutor;
120126

@@ -252,6 +258,9 @@ CompletableFuture<Void> enqueue(
252258
private CompletableFuture<Void> notifyWatchesInternal(long newRevision, List<Entry> updatedEntries, HybridTimestamp time) {
253259
assert time != null;
254260

261+
Set<Long> revisionsSet = updatedEntries.stream().map(Entry::revision).collect(Collectors.toUnmodifiableSet());
262+
assert revisionsSet.size() <= 1 : "Update entries are associated with different revisions, revisions=" + revisionsSet;
263+
255264
List<Entry> filteredUpdatedEntries = updatedEntries.isEmpty() ? emptyList() : updatedEntries.stream()
256265
.filter(WatchProcessor::isNotIdempotentCacheCommand)
257266
.collect(toList());
@@ -274,7 +283,7 @@ private CompletableFuture<Void> notifyWatchesInternal(long newRevision, List<Ent
274283
return newNotificationFuture;
275284
}, newNotificationFuture -> {
276285
invokeNotificationFutureListeners(newNotificationFuture, filteredUpdatedEntries, time);
277-
}, updatedEntriesKeysInfo(updatedEntries));
286+
}, updatedEntriesKeysInfo(newRevision, updatedEntries));
278287
}
279288

280289
private void invokeNotificationFutureListeners(
@@ -287,10 +296,11 @@ private void invokeNotificationFutureListeners(
287296
}
288297
}
289298

290-
private static Supplier<String> updatedEntriesKeysInfo(List<Entry> updatedEntries) {
299+
private Supplier<String> updatedEntriesKeysInfo(long revision, List<Entry> updatedEntries) {
291300
return () -> updatedEntries.stream()
292301
.map(entry -> new String(entry.key(), UTF_8))
293-
.collect(joining(",", "Keys of updated entries: ", ""));
302+
.collect(joining(", ", "Keys of revision: " + revision + " and previous revision: " + this.revision
303+
+ "with updated entries: ", ""));
294304
}
295305

296306
private static CompletableFuture<Void> performWatchesNotifications(
@@ -403,6 +413,8 @@ private void invokeOnRevisionCallback(long revision, HybridTimestamp time) {
403413
watchEventHandlingCallback.onSafeTimeAdvanced(time);
404414

405415
watchEventHandlingCallback.onRevisionApplied(revision);
416+
417+
this.revision = revision;
406418
}
407419

408420
/**

modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -4199,6 +4199,7 @@ public void testChangePeersAndLearnersChaosWithSnapshot() throws Exception {
41994199
}
42004200

42014201
@Test
4202+
@Disabled("https://issues.apache.org/jira/browse/IGNITE-26249")
42024203
public void testChangePeersAndLearnersChaosWithoutSnapshot() throws Exception {
42034204
// start cluster
42044205
List<TestPeer> peers = new ArrayList<>();
@@ -4246,6 +4247,7 @@ public void testChangePeersAndLearnersChaosWithoutSnapshot() throws Exception {
42464247
}
42474248

42484249
@Test
4250+
@Disabled("https://issues.apache.org/jira/browse/IGNITE-26249")
42494251
public void testChangePeersAndLearnersChaosApplyTasks() throws Exception {
42504252
// start cluster
42514253
List<TestPeer> peers = new ArrayList<>();

0 commit comments

Comments
 (0)