Skip to content

Fixed multiple configNode bugs#17609

Open
Caideyipi wants to merge 1 commit intomasterfrom
config-bug
Open

Fixed multiple configNode bugs#17609
Caideyipi wants to merge 1 commit intomasterfrom
config-bug

Conversation

@Caideyipi
Copy link
Copy Markdown
Collaborator

Description

  1. In Simple mode, a successful response will be returned stating "Metadata has taken effect, but WAL has not been flushed to disk." Configuration changes will be lost after a restart.
  2. In the Simple mode, some files have the same sorting key, posing a risk of log replay in disorder.
  3. The return TSStatus of ConsensusManager.write() is not checked, and the exception is only a warn followed by proceeding to the success branch. This means that if consensus writing is rejected, redirected, or the state machine returns a failure, the caller will still receive ACCEPT, while the local heartbeat cache/metrics have already been created in advance, leading to a divergence between metadata and runtime states.
  4. The value returned by getRegionGroupCount() increases by 1 every time it is counted, and this value is used for capacity expansion judgment, display, and maxRegionGroupNum calculation, which directly affects the region scaling decision.
  5. When taking a snapshot, the regionMaintainTaskList lock is not acquired before traversing. Concurrent offer/poll may throw ConcurrentModificationException or put the maintenance queue snapshot into an inconsistent state.
  6. If the pipe listener fails to process the snapshot, the entire takeSnapshot() method will return false; however, ConfigRegionStateMachine combines and absorbs the failures of loadSnapshot and the pipe listener, which can amplify auxiliary link failures into consensus snapshot failures, and even mask the real failure of snapshot loading.

This PR has:

  • been self-reviewed.
    • concurrent read
    • concurrent write
    • concurrent read and write
  • added documentation for new or modified features or behaviors.
  • added Javadocs for most classes and all non-trivial methods.
  • added or updated version, license, or notice information
  • added comments explaining the "why" and the intent of the code wherever would not be obvious
    for an unfamiliar reader.
  • added unit tests or modified existing tests to cover new code paths, ensuring the threshold
    for code coverage.
  • added integration tests.
  • been tested in a test IoTDB cluster.

Key changed/added classes (or packages if there are too many classes) in this PR

@sonarqubecloud
Copy link
Copy Markdown

sonarqubecloud Bot commented May 6, 2026

@codecov
Copy link
Copy Markdown

codecov Bot commented May 6, 2026

Codecov Report

❌ Patch coverage is 32.36715% with 140 lines in your changes missing coverage. Please review.
✅ Project coverage is 40.09%. Comparing base (6042c4f) to head (2047d3a).
⚠️ Report is 5 commits behind head on master.

Files with missing lines Patch % Lines
...che/iotdb/confignode/manager/node/NodeManager.java 42.30% 75 Missing ⚠️
...nsensus/statemachine/ConfigRegionStateMachine.java 9.23% 59 Missing ⚠️
...gnode/persistence/executor/ConfigPlanExecutor.java 0.00% 2 Missing ⚠️
...onfignode/persistence/partition/PartitionInfo.java 71.42% 2 Missing ⚠️
...de/procedure/impl/node/AddConfigNodeProcedure.java 0.00% 2 Missing ⚠️
Additional details and impacted files
@@             Coverage Diff              @@
##             master   #17609      +/-   ##
============================================
+ Coverage     39.94%   40.09%   +0.14%     
  Complexity     2554     2554              
============================================
  Files          5176     5176              
  Lines        348546   348658     +112     
  Branches      44564    44586      +22     
============================================
+ Hits         139242   139802     +560     
+ Misses       209304   208856     -448     

☔ View full report in Codecov by Sentry.
📢 Have feedback on the report? Share it here.

🚀 New features to boost your workflow:
  • ❄️ Test Analytics: Detect flaky tests, report on failures, and find test suite problems.

Copy link
Copy Markdown
Contributor

@CRZbulabula CRZbulabula left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Overall

This PR fixes 6 real correctness bugs in ConfigNode (WAL ordering, log sorting, unchecked consensus writes, off-by-one, concurrent snapshot, pipe-listener failure masking). Changes are well-structured, rollback logic is carefully designed, and test coverage for core scenarios is solid.

Inline comments below cover a few items worth attention before merge.

if (persistStatus.getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
return persistStatus;
}
}
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

WAL replay idempotency requirement

With the new write-ahead ordering (persist → execute), if persistPlanForSimpleConsensus succeeds at line 125 but executor.executeNonQueryPlan fails at line 133, the plan is already persisted in WAL and will be replayed on restart.

This is correct WAL semantics, but it introduces a hard requirement: all ConfigPhysicalPlan implementations must be idempotent under replay. For example, a RegisterDataNodePlan replayed against an already-registered node must not fail or double-register.

Is this idempotency property currently guaranteed across all ConfigPhysicalPlan types? If not, this could cause issues on crash recovery in edge cases.

ByteBuffer buffer = plan.serializeToByteBuffer();
buffer.position(buffer.limit());
simpleLogWriter.write(buffer);
simpleLogWriter.force();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

force() on every write makes scheduled flush redundant

Now that every write is synchronously forced here, the scheduled flushWALForSimpleConsensus thread (lines 447-455) becomes redundant — it calls simpleLogWriter.force() periodically, but there's nothing left to flush.

This is harmless (double-force is idempotent) but worth either:

  1. Removing the scheduled thread to avoid confusion, or
  2. Adding a brief comment explaining the intentional belt-and-suspenders approach.

For ConfigNode metadata operations (low write frequency), the per-write force() is the right durability choice.

"Persist current ConfigPhysicalPlan for ConfigNode SimpleConsensus mode failed", e);
return new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode())
.setMessage(
"Persist ConfigNode SimpleConsensus log failed: " + String.valueOf(e.getMessage()));
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: String.valueOf(e.getMessage()) is redundant — string concatenation with + already handles null by converting to "null". Can simplify to:

"Persist ConfigNode SimpleConsensus log failed: " + e.getMessage()

final List<RegionMaintainTask> copiedRegionMaintainTaskList;
synchronized (regionMaintainTaskList) {
copiedRegionMaintainTaskList = new ArrayList<>(regionMaintainTaskList);
}
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good fix — synchronized copy before iterating. One minor observation: this copies the entire list under the lock. If regionMaintainTaskList can grow very large, this could cause a brief pause. In practice, for ConfigNode this list is typically small, so this is fine. Just noting for future awareness.

Copy link
Copy Markdown
Contributor

@CRZbulabula CRZbulabula left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

PTAL.

}
});
return result.getAndIncrement();
return result.get();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this a bug?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants