-
Notifications
You must be signed in to change notification settings - Fork 569
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Corrupt partition prevents startup of node #10451
Comments
@megglos maybe you can attach also the data for the partition |
I added the partition logs at least, the snapshot is 8-9GB. I'm downloading it now and will put it in a drive. |
I've added the severity to be critical if we really suspect corruption, as imo that would be a release blocker (at least until we figure out whether this is already present in the latest release or if its something we introduced with 8.1) Don't hesitate to reach out for more 👀 and 🧠 to solve this |
Agree here with @npepinpe |
I guess saving the logs would be also great. |
I can take a look at it. if no one is. I'm waiting for my PRs to merge anyway 🤞 |
Here is what we observe in the log Partition two starts receiving snapshot from the leader.
Then it resets the log, because it has to throw away the log when a follower receives a snapshot.
Immediately, the broker is shutting down.
But we don't see a log that says snapshot After restart, we see that the snapshot is not committed. So it is deleted and snapshot store loads the previous snapshot. (which is the correct behavior).
But at this time the journal is at index |
Yikes, so we threw away the log before the snapshot was committed? :s It seems we do this to avoid an existing edge case where we have a snapshot but the log was not reset. So I guess we need to make reset + commit somehow atomic 😄 |
Thanks for checking @deepthidevaki !
|
Resetting the log before snapshot is committed was added recently in this PR. Assumption was that, if the system crashes before snapshot is committed, it will start with an empty state. This is ok because we can handle node restarts with data loss. But even though journal is empty, it's index is updated. This is why the check fails. |
This would be difficult to achieve, I assume 😄
This will re-introduce the edge case that we tried to fix, no? 🤔 I'm thinking about the following fix: Would this hide any of the valid corruption cases? |
Yes. Manual recovery is possible in this case. But since this is not actually a corruption, but a bug, we should avoid this. |
I don't see any issues with this approach but since I also missed this in the initial review of #10183 I'm not overly confident in my assessment so a second opinion would be great 👀. |
Let's look at different scenarios, where the snapshot is actually corrupted after they have been successfully committed.
|
In summary, this is one case where we won't be able to detect the corrupted state. But this can happen even now. |
10443: Do not take a backup if it already exists r=deepthidevaki a=deepthidevaki ## Description After restore, the log is truncated to the checkpoint position. So the checkpoint record is processed again and will trigger a new backup with the same Id of the backup it restored from. With this PR, `BackupService` handles this case gracefully. In addition, we also do not take a new backup if existing backup is failed or in progress. Alternatively, we can delete this backup and take a new one. But chances of it happening (i.e triggering a new backup when one already is in progress/failed) is very low. So we can keep this simple. ## Related issues closes #10430 10450: fix(raft): handle exceptions on partition server init r=megglos a=megglos ## Description Previously any RuntimeException happening in RaftPartitionServer#initServer lead to a broken future chain during start which lead to a stale node without any logs on the actual exception occurred during init. Ultimately flying silently till [here](https://github.com/camunda/zeebe/blob/main/broker/src/main/java/io/camunda/zeebe/broker/bootstrap/PartitionManagerStep.java#L42) bringing the startup to a halt. With this change issues are transparent, see this [log](https://console.cloud.google.com/logs/query;cursorTimestamp=2022-09-22T11:20:44.904454673Z;query=resource.labels.namespace_name%3D%22medic-cw-37-de38e9e086-benchmark-mixed%22%0Aresource.labels.pod_name%3D%22medic-cw-37-de38e9e086-benchmark-mixed-zeebe-2%22%0A-resource.labels.container_name%3D%22debugger-9q4tw%22%0A-logName%3D%22projects%2Fzeebe-io%2Flogs%2Fevents%22%0Atimestamp%3D%222022-09-22T11:20:44.904454673Z%22%0AinsertId%3D%2238ntsbk0c2ikn344%22%0Atimestamp%3D%222022-09-22T11:20:44.904454673Z%22%0AinsertId%3D%2238ntsbk0c2ikn344%22;summaryFields=:false:32:beginning;timeRange=2022-09-22T10:20:44.905Z%2F2022-09-22T11:20:44.905Z?project=zeebe-io) from a pod created with this change. This bug was hiding the underlying issue a node not being able to start due to #10451 . ## Related issues relates to #10451 10458: Reorganize stream processor and engine tests r=Zelldon a=Zelldon ## Description Moved some tests around to make it easier to detect which need to be migrated for #10455 and to make it easier to create the new module and copy the tests, which are part of the StreamProcessor see #10130 <!-- Please explain the changes you made here. --> ## Related issues <!-- Which issues are closed by this PR or are related --> related to #10455 related to #10130 Co-authored-by: Deepthi Devaki Akkoorath <deepthidevaki@gmail.com> Co-authored-by: Meggle (Sebastian Bathke) <sebastian.bathke@camunda.com> Co-authored-by: Christopher Zell <zelldon91@googlemail.com>
10450: fix(raft): handle exceptions on partition server init r=megglos a=megglos ## Description Previously any RuntimeException happening in RaftPartitionServer#initServer lead to a broken future chain during start which lead to a stale node without any logs on the actual exception occurred during init. Ultimately flying silently till [here](https://github.com/camunda/zeebe/blob/main/broker/src/main/java/io/camunda/zeebe/broker/bootstrap/PartitionManagerStep.java#L42) bringing the startup to a halt. With this change issues are transparent, see this [log](https://console.cloud.google.com/logs/query;cursorTimestamp=2022-09-22T11:20:44.904454673Z;query=resource.labels.namespace_name%3D%22medic-cw-37-de38e9e086-benchmark-mixed%22%0Aresource.labels.pod_name%3D%22medic-cw-37-de38e9e086-benchmark-mixed-zeebe-2%22%0A-resource.labels.container_name%3D%22debugger-9q4tw%22%0A-logName%3D%22projects%2Fzeebe-io%2Flogs%2Fevents%22%0Atimestamp%3D%222022-09-22T11:20:44.904454673Z%22%0AinsertId%3D%2238ntsbk0c2ikn344%22%0Atimestamp%3D%222022-09-22T11:20:44.904454673Z%22%0AinsertId%3D%2238ntsbk0c2ikn344%22;summaryFields=:false:32:beginning;timeRange=2022-09-22T10:20:44.905Z%2F2022-09-22T11:20:44.905Z?project=zeebe-io) from a pod created with this change. This bug was hiding the underlying issue a node not being able to start due to #10451 . ## Related issues relates to #10451 10458: Reorganize stream processor and engine tests r=Zelldon a=Zelldon ## Description Moved some tests around to make it easier to detect which need to be migrated for #10455 and to make it easier to create the new module and copy the tests, which are part of the StreamProcessor see #10130 <!-- Please explain the changes you made here. --> ## Related issues <!-- Which issues are closed by this PR or are related --> related to #10455 related to #10130 Co-authored-by: Meggle (Sebastian Bathke) <sebastian.bathke@camunda.com> Co-authored-by: Christopher Zell <zelldon91@googlemail.com>
10443: Do not take a backup if it already exists r=deepthidevaki a=deepthidevaki ## Description After restore, the log is truncated to the checkpoint position. So the checkpoint record is processed again and will trigger a new backup with the same Id of the backup it restored from. With this PR, `BackupService` handles this case gracefully. In addition, we also do not take a new backup if existing backup is failed or in progress. Alternatively, we can delete this backup and take a new one. But chances of it happening (i.e triggering a new backup when one already is in progress/failed) is very low. So we can keep this simple. ## Related issues closes #10430 10463: Do not fail consistency check if log is empty r=deepthidevaki a=deepthidevaki ## Description When a follower receives a snapshot from the leader, it has to throw away the log and reset the log to `snapshotIndex + 1`. Previously we were doing it in the following order: 1. commit snapshot 2. reset In this case, if the system crashed after step 1, when the node restarts it is in an invalid state because the log is not reset after the snapshot. To prevent this case, we reset the log on startup based on the existing snapshot. This was buggy and caused issues, which was fixed by #10183. The fix was to reverse the order: 1. reset log 2. commit snapshot. So on restart, there is no need to reset the log. If the system crashes after step 1, we have any empty log and no snapshot (or a previous snapshot). This is a valid state because this follower is not in the quorum, so no data is lost. After the restart the follower will receive the snapshot and the following events. But this caused the consistency check to fail because it detected gaps between the snapshot and the first log entry. The state is not actually inconsistent, because no data is lost. So we fix this by updating the consistency check to treat this is a valid state. To make the state valid, if the log is empty, we reset it based on the available snapshot. ## Related issues closes #10451 Co-authored-by: Deepthi Devaki Akkoorath <deepthidevaki@gmail.com>
10450: fix(raft): handle exceptions on partition server init r=megglos a=megglos ## Description Previously any RuntimeException happening in RaftPartitionServer#initServer lead to a broken future chain during start which lead to a stale node without any logs on the actual exception occurred during init. Ultimately flying silently till [here](https://github.com/camunda/zeebe/blob/main/broker/src/main/java/io/camunda/zeebe/broker/bootstrap/PartitionManagerStep.java#L42) bringing the startup to a halt. With this change issues are transparent, see this [log](https://console.cloud.google.com/logs/query;cursorTimestamp=2022-09-22T11:20:44.904454673Z;query=resource.labels.namespace_name%3D%22medic-cw-37-de38e9e086-benchmark-mixed%22%0Aresource.labels.pod_name%3D%22medic-cw-37-de38e9e086-benchmark-mixed-zeebe-2%22%0A-resource.labels.container_name%3D%22debugger-9q4tw%22%0A-logName%3D%22projects%2Fzeebe-io%2Flogs%2Fevents%22%0Atimestamp%3D%222022-09-22T11:20:44.904454673Z%22%0AinsertId%3D%2238ntsbk0c2ikn344%22%0Atimestamp%3D%222022-09-22T11:20:44.904454673Z%22%0AinsertId%3D%2238ntsbk0c2ikn344%22;summaryFields=:false:32:beginning;timeRange=2022-09-22T10:20:44.905Z%2F2022-09-22T11:20:44.905Z?project=zeebe-io) from a pod created with this change. This bug was hiding the underlying issue a node not being able to start due to #10451 . ## Related issues relates to #10451 Co-authored-by: Meggle (Sebastian Bathke) <sebastian.bathke@camunda.com>
10463: Do not fail consistency check if log is empty r=deepthidevaki a=deepthidevaki ## Description When a follower receives a snapshot from the leader, it has to throw away the log and reset the log to `snapshotIndex + 1`. Previously we were doing it in the following order: 1. commit snapshot 2. reset In this case, if the system crashed after step 1, when the node restarts it is in an invalid state because the log is not reset after the snapshot. To prevent this case, we reset the log on startup based on the existing snapshot. This was buggy and caused issues, which was fixed by #10183. The fix was to reverse the order: 1. reset log 2. commit snapshot. So on restart, there is no need to reset the log. If the system crashes after step 1, we have any empty log and no snapshot (or a previous snapshot). This is a valid state because this follower is not in the quorum, so no data is lost. After the restart the follower will receive the snapshot and the following events. But this caused the consistency check to fail because it detected gaps between the snapshot and the first log entry. The state is not actually inconsistent, because no data is lost. So we fix this by updating the consistency check to treat this is a valid state. To make the state valid, if the log is empty, we reset it based on the available snapshot. ## Related issues closes #10451 10482: deps(maven): bump snakeyaml from 1.32 to 1.33 r=Zelldon a=dependabot[bot] Bumps [snakeyaml](https://bitbucket.org/snakeyaml/snakeyaml) from 1.32 to 1.33. <details> <summary>Commits</summary> <ul> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/eafb23ec31a0babe591c00e1b50e557a5e3f9a1d"><code>eafb23e</code></a> [maven-release-plugin] prepare for next development iteration</li> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/26624702fab8e0a1c301d7fad723c048528f75c3"><code>2662470</code></a> Improve JavaDoc</li> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/80827798f06aeb3d4f2632b94075ca7633418829"><code>8082779</code></a> Always emit numberish strings with quotes</li> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/42d6c79430431fe9033d3ba50f6a7dc6798ba7ad"><code>42d6c79</code></a> Reformat test</li> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/1962a437263348c3b90857cda4bbfa2bd97908f8"><code>1962a43</code></a> Refactor: rename variables in Emitter</li> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/bc594ad6e2b87c3fc26844e407276796fd866a40"><code>bc594ad</code></a> Issue 553: honor code point limit in loadAll</li> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/c3e98fd755a949f65cf11f2ff39e55a1c2afd1c2"><code>c3e98fd</code></a> Update changes.xml</li> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/a06f76859f2f07580b1d9fa6b66ea84aaad26cf8"><code>a06f768</code></a> Remove deprecated Tag manipulation</li> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/5a0027a3781b92f59bf92cdeb1b7590589993efd"><code>5a0027a</code></a> Remove unused WhitespaceToken</li> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/3f05838828b8df36ab961bf836f373b8c20cb8ff"><code>3f05838</code></a> Improve JavaDoc</li> <li>Additional commits viewable in <a href="https://bitbucket.org/snakeyaml/snakeyaml/branches/compare/snakeyaml-1.33..snakeyaml-1.32">compare view</a></li> </ul> </details> <br /> [![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=org.yaml:snakeyaml&package-manager=maven&previous-version=1.32&new-version=1.33)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting ``@dependabot` rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) --- <details> <summary>Dependabot commands and options</summary> <br /> You can trigger Dependabot actions by commenting on this PR: - ``@dependabot` rebase` will rebase this PR - ``@dependabot` recreate` will recreate this PR, overwriting any edits that have been made to it - ``@dependabot` merge` will merge this PR after your CI passes on it - ``@dependabot` squash and merge` will squash and merge this PR after your CI passes on it - ``@dependabot` cancel merge` will cancel a previously requested merge and block automerging - ``@dependabot` reopen` will reopen this PR if it is closed - ``@dependabot` close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - ``@dependabot` ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - ``@dependabot` ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - ``@dependabot` ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself) </details> 10486: test: `PartitionRestoreServiceTest` does not block on taking a backup r=oleschoenburg a=oleschoenburg We saw some unit tests timing out in `PartitionRestoreServiceTest`: ``` "ForkJoinPool-1-worker-1" #19 daemon prio=5 os_prio=0 cpu=1567.91ms elapsed=914.45s tid=0x00007facfca78b60 nid=0x15ab5 waiting on condition [0x00007facb83df000] java.lang.Thread.State: WAITING (parking) at jdk.internal.misc.Unsafe.park(java.base@17.0.4.1/Native Method) - parking to wait for <0x0000000511f04c68> (a java.util.concurrent.CompletableFuture$Signaller) at java.util.concurrent.locks.LockSupport.park(java.base@17.0.4.1/LockSupport.java:211) at java.util.concurrent.CompletableFuture$Signaller.block(java.base@17.0.4.1/CompletableFuture.java:1864) at java.util.concurrent.ForkJoinPool.compensatedBlock(java.base@17.0.4.1/ForkJoinPool.java:3449) at java.util.concurrent.ForkJoinPool.managedBlock(java.base@17.0.4.1/ForkJoinPool.java:3432) at java.util.concurrent.CompletableFuture.waitingGet(java.base@17.0.4.1/CompletableFuture.java:1898) at java.util.concurrent.CompletableFuture.join(java.base@17.0.4.1/CompletableFuture.java:2117) at io.camunda.zeebe.restore.PartitionRestoreServiceTest.takeBackup(PartitionRestoreServiceTest.java:212) at io.camunda.zeebe.restore.PartitionRestoreServiceTest.shouldFailToRestoreWhenSnapshotIsCorrupted(PartitionRestoreServiceTest.java:182) ``` With these changes here we ensure that the test does not wait forever on a backup, instead setting a maximum of 30 seconds. Additionally, `TestRestorableBackupStore` now fails the future when a backup is marked as failed. 10489: Do not use DefaultActorClock r=Zelldon a=Zelldon ## Description The default ActorClock is not thread safe and shouldn't be shared with multiple threads. This means we need to set the clock in the ActorClockConfiguration to null. Creating the ActorScheduler with no clock will cause that each threads gets its own clock. Note: This is a quick fix, at some point, we want to make DefaultActorClock threadsafe so we can use always the same clock. See #10400 <!-- Please explain the changes you made here. --> ## Related issues <!-- Which issues are closed by this PR or are related --> related #10400 10490: ci(macos): set code cache size of 64m r=megglos a=megglos To counter occasional out of code cache errors observed on macos builds. Co-authored-by: Deepthi Devaki Akkoorath <deepthidevaki@gmail.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Ole Schönburg <ole.schoenburg@gmail.com> Co-authored-by: Christopher Zell <zelldon91@googlemail.com> Co-authored-by: Meggle (Sebastian Bathke) <sebastian.bathke@camunda.com>
10463: Do not fail consistency check if log is empty r=deepthidevaki a=deepthidevaki ## Description When a follower receives a snapshot from the leader, it has to throw away the log and reset the log to `snapshotIndex + 1`. Previously we were doing it in the following order: 1. commit snapshot 2. reset In this case, if the system crashed after step 1, when the node restarts it is in an invalid state because the log is not reset after the snapshot. To prevent this case, we reset the log on startup based on the existing snapshot. This was buggy and caused issues, which was fixed by #10183. The fix was to reverse the order: 1. reset log 2. commit snapshot. So on restart, there is no need to reset the log. If the system crashes after step 1, we have any empty log and no snapshot (or a previous snapshot). This is a valid state because this follower is not in the quorum, so no data is lost. After the restart the follower will receive the snapshot and the following events. But this caused the consistency check to fail because it detected gaps between the snapshot and the first log entry. The state is not actually inconsistent, because no data is lost. So we fix this by updating the consistency check to treat this is a valid state. To make the state valid, if the log is empty, we reset it based on the available snapshot. ## Related issues closes #10451 10482: deps(maven): bump snakeyaml from 1.32 to 1.33 r=Zelldon a=dependabot[bot] Bumps [snakeyaml](https://bitbucket.org/snakeyaml/snakeyaml) from 1.32 to 1.33. <details> <summary>Commits</summary> <ul> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/eafb23ec31a0babe591c00e1b50e557a5e3f9a1d"><code>eafb23e</code></a> [maven-release-plugin] prepare for next development iteration</li> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/26624702fab8e0a1c301d7fad723c048528f75c3"><code>2662470</code></a> Improve JavaDoc</li> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/80827798f06aeb3d4f2632b94075ca7633418829"><code>8082779</code></a> Always emit numberish strings with quotes</li> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/42d6c79430431fe9033d3ba50f6a7dc6798ba7ad"><code>42d6c79</code></a> Reformat test</li> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/1962a437263348c3b90857cda4bbfa2bd97908f8"><code>1962a43</code></a> Refactor: rename variables in Emitter</li> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/bc594ad6e2b87c3fc26844e407276796fd866a40"><code>bc594ad</code></a> Issue 553: honor code point limit in loadAll</li> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/c3e98fd755a949f65cf11f2ff39e55a1c2afd1c2"><code>c3e98fd</code></a> Update changes.xml</li> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/a06f76859f2f07580b1d9fa6b66ea84aaad26cf8"><code>a06f768</code></a> Remove deprecated Tag manipulation</li> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/5a0027a3781b92f59bf92cdeb1b7590589993efd"><code>5a0027a</code></a> Remove unused WhitespaceToken</li> <li><a href="https://bitbucket.org/snakeyaml/snakeyaml/commits/3f05838828b8df36ab961bf836f373b8c20cb8ff"><code>3f05838</code></a> Improve JavaDoc</li> <li>Additional commits viewable in <a href="https://bitbucket.org/snakeyaml/snakeyaml/branches/compare/snakeyaml-1.33..snakeyaml-1.32">compare view</a></li> </ul> </details> <br /> [![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=org.yaml:snakeyaml&package-manager=maven&previous-version=1.32&new-version=1.33)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting ``@dependabot` rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) --- <details> <summary>Dependabot commands and options</summary> <br /> You can trigger Dependabot actions by commenting on this PR: - ``@dependabot` rebase` will rebase this PR - ``@dependabot` recreate` will recreate this PR, overwriting any edits that have been made to it - ``@dependabot` merge` will merge this PR after your CI passes on it - ``@dependabot` squash and merge` will squash and merge this PR after your CI passes on it - ``@dependabot` cancel merge` will cancel a previously requested merge and block automerging - ``@dependabot` reopen` will reopen this PR if it is closed - ``@dependabot` close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - ``@dependabot` ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - ``@dependabot` ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - ``@dependabot` ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself) </details> Co-authored-by: Deepthi Devaki Akkoorath <deepthidevaki@gmail.com> Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
10494: [Backport stable/8.0] Do not fail consistency check if log is empty r=deepthidevaki a=backport-action # Description Backport of #10463 to `stable/8.0`. relates to #10451 10540: [Backport stable/8.0] Ensure retries are not interleaved even on multiple sequential calls r=deepthidevaki a=npepinpe ## Description This PR backports part of the changes found in #10289, notably the ones in the scheduler about the retry strategies (skipping the engine ones which are all around new 8.1.0 code). It seems I forgot to backport that part, as we did remove `runUntilDone` in 8.0 as well, so we should fix the retry strategies. ## Related issues backports #10289 Co-authored-by: Deepthi Devaki Akkoorath <deepthidevaki@gmail.com> Co-authored-by: Nicolas Pepin-Perreault <nicolas.pepin-perreault@camunda.com>
10494: [Backport stable/8.0] Do not fail consistency check if log is empty r=deepthidevaki a=backport-action # Description Backport of #10463 to `stable/8.0`. relates to #10451 10576: [Backport stable/8.0] Add verification timeout to SuccessfulDeploymentTest.shouldSendResponse r=saig0 a=backport-action # Description Backport of #10573 to `stable/8.0`. relates to #10492 Co-authored-by: Deepthi Devaki Akkoorath <deepthidevaki@gmail.com> Co-authored-by: Christopher Zell <zelldon91@googlemail.com>
Describe the bug
A benchmark node became unhealthy after a restart because of being stuck at starting the Partition Server for partition 2.
To Reproduce
Zeebe node 2 in
medic-cw-37-de38e9e086-benchmark
is stuck at this issue.Expected behavior
Would it make sense to discard the log and/or snapshot in this case to recover?
Log/Stacktrace
Full Stacktrace
Environment:
Partition logs:
partition-2-logs.tar.gz
The text was updated successfully, but these errors were encountered: