Skip to content

Replace custom Raft protocol with Apache Ratis#3798

Open
lvca wants to merge 88 commits intomainfrom
apache-ratis
Open

Replace custom Raft protocol with Apache Ratis#3798
lvca wants to merge 88 commits intomainfrom
apache-ratis

Conversation

@lvca
Copy link
Copy Markdown
Member

@lvca lvca commented Apr 5, 2026

This comes from @robfrank after working into improving HA for many months with poor results. The main reason was the RAFT implementation itself I created years ago (my fault) that was quite limited and poorly tested. In order to reach the level of rock-solidity we want, it might take 1 year or more. So @robfrank had the idea to replace our internal RAFT system with Apache Ratis 3.2.2 - a battle-tested, formally correct implementation of the Raft consensus protocol used in production by Apache Ozone, IoTDB, and Alluxio.

After reviewing Apache Ratis internal architecture, I started from scratch with the integration to go in parallel with @robfrank branch ha-redesign. I've taken from @robfrank's branch ha-redesign many tests and some components.

This change is transparent to users - the HTTP API, database API, query languages, and client libraries remain unchanged. The internal HA protocol now uses gRPC (shaded by Ratis) instead of custom TCP binary messages.

92 files changed, +7,917 / -6,639 lines (net +1,278)

What was removed (34 files)

The entire custom HA stack: HAServer, Leader2ReplicaNetworkExecutor, Replica2LeaderNetworkExecutor, LeaderNetworkListener, ReplicationLogFile, ReplicationProtocol, 21 message classes (TxRequest, CommandForwardRequest, etc.), and related infrastructure.

What was added (22 new files)

Core HA (5 production files in server/ha/ratis/):

  • RaftHAServer - Ratis server lifecycle, gRPC transport, peer management, dynamic membership
  • ArcadeDBStateMachine - Ratis state machine for WAL replication with election tracking
  • RaftLogEntry - binary serialization for Raft log entries (TRANSACTION, TRANSACTION_FORWARD)
  • HALog - verbose logging utility (arcadedb.ha.logVerbose=0/1/2/3)
  • SnapshotHttpHandler - HTTP endpoint for database snapshot serving
  • ClusterMonitor - replication lag monitoring with configurable warning threshold

Tests (10 files):

  • RaftLogEntryTest (4 unit tests) - serialization round-trip
  • RaftHAServerIT (3 tests) - raw Ratis consensus
  • RaftReplicationIT (5 tests) - full cluster replication
  • RaftHAComprehensiveIT (17 tests) - data consistency, failover, concurrent writes, schema changes, rolling upgrade, large transactions
  • ReadConsistencyIT (3 tests) - EVENTUAL, READ_YOUR_WRITES, LINEARIZABLE
  • ClusterTokenAuthIT (5 tests) - cluster-internal auth validation
  • ClusterMonitorTest (5 unit tests) - lag tracking

Docker e2e tests (7 files, tagged e2e-ha, require Docker):

  • HAReplicationE2ETest - basic replication, leader failover, follower proxy
  • HARollingRestartE2ETest - rolling restart with continuous writes
  • HANetworkPartitionE2ETest - follower isolation via Docker network disconnect
  • HANetworkDelayE2ETest - 200ms-2000ms latency injection via Toxiproxy
  • HAPacketLossE2ETest - 5%-50% packet loss injection via Toxiproxy

Key features

  • Apache Ratis consensus - pre-vote protocol, parallel voting, gRPC streaming, leader lease
  • Read consistency levels - EVENTUAL (read locally), READ_YOUR_WRITES (default, bookmark-based), LINEARIZABLE (wait for all committed writes). Configurable per-connection via RemoteDatabase.setReadConsistency() or globally via arcadedb.ha.readConsistency
  • Cluster token auth - deterministic shared secret derived from clusterName + rootPassword for inter-node HTTP forwarding. Replaces Basic auth forwarding for session-based auth
  • K8s automation - auto-join on scale-up (tryAutoJoinCluster), auto-leave on scale-down (leaveCluster in preStop hook)
  • HA management commands - ha add/remove peer, ha transfer leader, ha step down, ha leave, ha verify database
  • Studio cluster dashboard - Overview (node cards, health badge), Metrics (replication lag chart with warning threshold, commit index, election count, uptime), Management (peer management, leadership transfer, database verification)
  • RemoteDatabase client failover - automatic retry with cluster topology reload on leader change (HTTP 503 -> NeedRetryException)
  • ClusterMonitor - background replication lag monitoring with configurable warning threshold (arcadedb.ha.replicationLagWarning)
  • gRPC channel refresh - RaftClient recreated on leader change to force fresh DNS resolution after network partitions
  • Snapshot via HTTP - follower catch-up via ZIP download from leader

New configuration settings

┌───────────────────────────────────┬──────────────────┬─────────────────────────────────────────────────────────┐
│              Setting              │     Default      │                       Description                       │
├───────────────────────────────────┼──────────────────┼─────────────────────────────────────────────────────────┤
│ arcadedb.ha.logVerbose            │ 0                │ HA verbose logging: 0=off, 1=basic, 2=detailed, 3=trace │
├───────────────────────────────────┼──────────────────┼─────────────────────────────────────────────────────────┤
│ arcadedb.ha.readConsistency       │ read_your_writes │ Default read consistency for follower reads             │
├───────────────────────────────────┼──────────────────┼─────────────────────────────────────────────────────────┤
│ arcadedb.ha.clusterToken          │ auto-derived     │ Shared secret for inter-node HTTP auth                  │
├───────────────────────────────────┼──────────────────┼─────────────────────────────────────────────────────────┤
│ arcadedb.ha.replicationLagWarning │ 1000             │ Raft log gap threshold for lag warnings (0=disabled)    │
└───────────────────────────────────┴──────────────────┴─────────────────────────────────────────────────────────┘

Test plan

  • RaftLogEntryTest - 4 unit tests
  • RaftHAServerIT - 3 pure Ratis consensus tests (3 nodes)
  • RaftReplicationIT - 5 cluster replication tests (3 nodes)
  • RaftHAComprehensiveIT - 17 comprehensive tests (3 nodes)
  • HTTP2ServersIT - 6 HTTP API tests (2 nodes)
  • ReplicationServerLeaderDownIT - leader failover with RemoteDatabase (3 nodes)
  • ReplicationServerLeaderChanges3TimesIT - 3 leader kill/restart cycles (3 nodes)
  • HASplitBrainIT - 5-node split brain (5 nodes)
  • ReadConsistencyIT - 3 read consistency level tests (2 nodes)
  • ClusterTokenAuthIT - 5 cluster auth tests (2 nodes)
  • ClusterMonitorTest - 5 unit tests
  • HAReplicationE2ETest - Docker e2e (requires Docker)
  • HANetworkDelayE2ETest - Toxiproxy latency (requires Docker)
  • HAPacketLossE2ETest - Toxiproxy packet loss (requires Docker)

@lvca lvca added this to the 26.4.1 milestone Apr 5, 2026
@lvca lvca self-assigned this Apr 5, 2026
@codacy-production
Copy link
Copy Markdown

codacy-production bot commented Apr 5, 2026

Not up to standards ⛔

🔴 Issues 6 medium · 26 minor

Alerts:
⚠ 32 issues (≤ 0 issues of at least minor severity)

Results:
32 new issues

Category Results
BestPractice 6 medium
2 minor
CodeStyle 24 minor

View in Codacy

🟢 Metrics 572 complexity

Metric Results
Complexity 572

View in Codacy

TIP This summary will be updated as you push new changes. Give us feedback

@claude
Copy link
Copy Markdown

claude bot commented Apr 5, 2026

Code Review

This is a substantial and well-motivated change - replacing ~6,000 lines of ad-hoc Raft-like code with Apache Ratis 3.2.1 is a significant improvement in correctness guarantees. The new test suite (36 tests including 17 comprehensive ones) is thorough. Below are issues worth addressing before merge.


Bugs

1. System.out.println in production code (HALog.java)

HALog.log() writes to both System.out AND LogManager. The System.out call must be removed per project conventions. It also logs every message twice (once formatted, once via the normal logger).

// HALog.java - remove this line:
System.out.println("[HA-" + level + "] " + formatted);

2. proxyToLeader - missing connection timeout

The conn (HttpURLConnection) in proxyToLeader() has no connect or read timeout set. If the leader is unreachable, this will hang the thread indefinitely. The SnapshotHttpHandler correctly sets timeouts (setConnectTimeout(30_000), setReadTimeout(300_000)); apply the same pattern here.

3. HALog format string - first replace() is a no-op

final String formatted = String.format(message.replace("%s", "%s").replace("%d", "%s"), args);

message.replace("%s", "%s") does nothing. If the intent is to normalize %d%s for String.format, that works, but the first replace is dead code and the logic is confusing. Consider removing it.

4. Spurious {} block in ServerIsNotTheLeaderException handler

In AbstractServerHttpHandler.handleRequest():

}
{        // <-- unnecessary anonymous block
  try {
    proxyToLeader(exchange, leaderAddr);
    return;
  } catch ...
}
sendErrorResponse(...)

The extra {...} block serves no purpose and looks like a copy-paste artefact.


Data Durability Concern

5. Leader commits locally before achieving quorum

replicateFromLeader() calls tx.commit2ndPhase(phase1) (durable local write) before raftHA.replicateTransaction(). If the leader crashes after local commit but before replication, the write is lost from the cluster's perspective even though the client received success.

Standard Raft commits only after quorum acknowledgement. This design is explicitly documented as intentional, but it weakens the durability guarantees. At a minimum, this should be prominently documented as a trade-off (the new docs/arcadedb-ha-26.4.1.md mentions it but could be clearer about the risk).


Security

6. Timing-attack-vulnerable cluster token comparison

// validateClusterForwardedAuth:
!expectedToken.equals(providedToken)

String.equals() is not constant-time. Use MessageDigest.isEqual() to prevent timing attacks on the shared cluster secret:

!MessageDigest.isEqual(expectedToken.getBytes(StandardCharsets.UTF_8), providedToken.getBytes(StandardCharsets.UTF_8))

7. User enumeration via error message

sendErrorResponse(exchange, 401, "Unknown forwarded user: " + forwardedUserValues.getFirst(), null, null);

Exposing the username in the response body of an auth failure enables user enumeration. Return a generic "Authentication failed" message instead.


Design / Maintainability

8. maxRetry silently mutated from caller's value

// RemoteHttpComponent:
if (maxRetry < 3)
  maxRetry = 3;

This overrides the caller's intent without signalling it. Callers passing maxRetry=1 for fast-fail scenarios will unexpectedly retry 3 times during elections. Consider using a local variable (effectiveRetry = Math.max(maxRetry, 3)) instead of mutating the parameter.

9. ATTRIBUTIONS.md not updated

Per CLAUDE.md: "When adding a dependency, you MUST update ATTRIBUTIONS.md and, if Apache-licensed with a NOTICE file, incorporate required notices into the main NOTICE file." Eight Ratis dependencies are added but ATTRIBUTIONS.md does not appear in the diff. Apache Ratis does have a NOTICE file.

10. ## TODO in committed documentation

docs/arcadedb-ha-26.4.1.md contains a ## TODO section listing future features. This is fine as a draft, but should either be removed or converted to GitHub issues before merging into the main branch.


Minor

  • The @deprecated note on getHA() / getRaftHA() alias in ArcadeDBServer is good, but @Deprecated annotation (with capital D) should be added to ensure IDE/compiler visibility.
  • RaftClusterStarter.java (test utility) uses System.out.println extensively - this is fine for a developer utility, though a comment clarifying it is a local dev tool (not executed by CI) would help.

Overall this is a high-quality implementation of a very complex change. The comprehensive test coverage and the transparent design documentation are appreciated. Addressing the data durability concern (#5) and the security issues (#6, #7) are the highest priority before merge.

Copy link
Copy Markdown
Contributor

@gemini-code-assist gemini-code-assist bot left a comment

Choose a reason for hiding this comment

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

Code Review

This pull request replaces the custom HA replication protocol with Apache Ratis, providing a formally verified Raft consensus implementation. The changes include a new Ratis state machine, snapshot management, and improved cluster monitoring. I have identified a flaw in the HTTP proxy's session-based authentication forwarding, a reliance on unreliable Thread.sleep in tests, and some unused code that should be cleaned up. Please address these issues to ensure robust authentication and reliable test execution.

Comment on lines +336 to +342
final String basicAuth = exchange.getAttachment(BASIC_AUTH_KEY);
if (basicAuth != null) {
// Extract username from stored Basic auth
final String decoded = new String(Base64.getDecoder().decode(
basicAuth.substring(AUTHORIZATION_BASIC.length() + 1)), java.nio.charset.StandardCharsets.UTF_8);
conn.setRequestProperty(HEADER_FORWARDED_USER, decoded.split(":")[0]);
}
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.

high

There appears to be a flaw in the logic for forwarding session-based authentication. This code attempts to get the original Basic auth from an attachment on the current exchange. However, if the request was authenticated with a session Bearer token, this attachment will not be present, and the forwarded user will not be set.

To fix this, you should retrieve the session object from the session token and get the user's name from there. This would be more robust and would not require attaching the basic auth string to the exchange.

Here's a suggested implementation:

      if (auth != null && auth.startsWith("Bearer AU-")) {
        // Session token: use cluster-internal auth headers instead
        conn.setRequestProperty(HEADER_CLUSTER_TOKEN, raftHA.getClusterToken());
        final String sessionToken = auth.substring(AUTHORIZATION_BEARER.length()).trim();
        final HttpAuthSession session = httpServer.getAuthSessionManager().getSession(sessionToken);
        if (session != null) {
            conn.setRequestProperty(HEADER_FORWARDED_USER, session.getUser().getName());
        } else {
            // Handle case where session is invalid or expired
        }
      } else if (auth != null) {
// ...

final JSONObject cluster = getClusterInfo(container);
if (cluster.has("isLeader") && cluster.getBoolean("isLeader"))
return true;
} catch (final Exception ignored) {}
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.

medium

Empty catch blocks should be avoided, even in tests, as they can swallow exceptions and hide underlying problems. It would be better to at least log the exception at a DEBUG or TRACE level. This would make debugging test failures easier. This applies to other similar empty catch blocks in this file and other new test files.

            } catch (final Exception e) {
              // Ignored during polling for leader
            }

Comment on lines +193 to +195
// Toxiproxy "slow_close" + "limit_data" simulate packet loss behavior.
// The "timeout" toxic with a probability achieves drop behavior.
// Using bandwidth toxic with very low rate to simulate packet-level loss.
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.

medium

The comment here is a bit misleading. It mentions slow_close, limit_data, and timeout toxics to simulate packet loss, but the implementation uses the bandwidth toxic. While limiting bandwidth can simulate some effects of packet loss (like timeouts), it's not the same as randomly dropping packets. The comment should be updated to accurately reflect the implementation, for example, by stating that it simulates packet loss effects by severely limiting bandwidth.

Comment on lines +663 to +696
private ResultSet forwardCommandToLeader(final String language, final String query, final Map<String, Object> namedParams,
final Object[] positionalParams) {
HALog.log(this, HALog.DETAILED, "Forwarding command to leader: %s %s (db=%s)", language, query, getName());

// Rollback the local transaction started by DatabaseAbstractHandler.transaction() wrapper.
// The command executes on the leader, so no local changes should be committed.
if (isTransactionActive())
rollback();

final RaftHAServer raftHA = server.getRaftHA();
final byte[] resultBytes = raftHA.forwardCommand(getName(), language, query, namedParams, positionalParams);
HALog.log(this, HALog.TRACE, "Command forwarded successfully: %d bytes result", resultBytes.length);

// Wait for the leader's WAL changes to be applied locally on this follower.
// Without this, a subsequent read on this server may not see the changes yet.
raftHA.waitForLocalApply();

// Check for error response
if (resultBytes.length > 0 && resultBytes[0] == 'E') {
final String error = new String(resultBytes, 1, resultBytes.length - 1);
throw new com.arcadedb.exception.CommandExecutionException(error);
}

// Deserialize binary result into ResultSet
final java.util.List<Map<String, Object>> rows =
com.arcadedb.server.ha.ratis.RaftLogEntry.deserializeCommandResult(resultBytes);
final com.arcadedb.query.sql.executor.InternalResultSet rs = new com.arcadedb.query.sql.executor.InternalResultSet();
for (final Map<String, Object> row : rows) {
final com.arcadedb.query.sql.executor.ResultInternal result = new com.arcadedb.query.sql.executor.ResultInternal(proxied);
result.setPropertiesFromMap(row);
rs.add(result);
}
return rs;
}
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.

medium

The private method forwardCommandToLeader seems to be unused. The logic in command() now throws ServerIsNotTheLeaderException, and the forwarding is handled by the HTTP proxy layer. This method appears to be a remnant of a different implementation strategy. To improve maintainability and avoid confusion, it should be removed if it's no longer needed.

.atMost(5, TimeUnit.MINUTES)
.pollInterval(1, TimeUnit.SECONDS)
.until(() -> getServer(serverNumber).getHA().getMessagesInQueue() == 0);
// With Ratis, replication is handled internally. Wait for state machine application.
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.

medium

Replacing the explicit wait for the replication queue to be empty with a fixed Thread.sleep() is unreliable and can lead to flaky tests. A better approach would be to implement a more deterministic wait. For example, you could get the leader's commit index and then poll the followers until their applied index matches the leader's commit index. This would ensure that replication is actually complete before proceeding with test assertions.

    // With Ratis, replication is handled internally. Await for state machine application on followers.
    final ArcadeDBServer leader = getLeaderServer();
    if (leader == null)
      return; // NO LEADER, CANNOT WAIT

    final long leaderCommitIndex = leader.getHA().getCommitIndex();

    for (int i = 0; i < getServerCount(); ++i) {
      final ArcadeDBServer server = getServer(i);
      if (server != leader)
        Awaitility.await().atMost(1, TimeUnit.MINUTES).until(() -> server.getHA().getLastAppliedIndex() >= leaderCommitIndex);
    }

@github-actions
Copy link
Copy Markdown
Contributor

github-actions bot commented Apr 5, 2026

📜 License Compliance Check

✅ License check passed. See artifacts for full report.

License Summary (first 50 lines)

Lists of 401 third-party dependencies.
     (Apache License 2.0) LZ4 Java Compression (at.yawk.lz4:lz4-java:1.10.4 - https://github.com/yawkat/lz4-java)
     (EPL 2.0) (GNU Lesser General Public License) Logback Classic Module (ch.qos.logback:logback-classic:1.5.32 - http://logback.qos.ch/logback-classic)
     (EPL 2.0) (GNU Lesser General Public License) Logback Core Module (ch.qos.logback:logback-core:1.5.32 - http://logback.qos.ch/logback-core)
     (Apache 2) ArcadeDB BOLT Protocol (com.arcadedb:arcadedb-bolt:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-bolt/)
     (Apache 2) ArcadeDB Console (com.arcadedb:arcadedb-console:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-console/)
     (Apache 2) ArcadeDB Engine (com.arcadedb:arcadedb-engine:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-engine/)
     (Apache 2) ArcadeDB GraphQL (com.arcadedb:arcadedb-graphql:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-graphql/)
     (Apache 2) ArcadeDB Gremlin (com.arcadedb:arcadedb-gremlin:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-gremlin/)
     (Apache 2) ArcadeDB gRPC Stubs (com.arcadedb:arcadedb-grpc:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-grpc/)
     (Apache 2) ArcadeDB gRPC Client (com.arcadedb:arcadedb-grpc-client:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-grpc-client/)
     (Apache 2) ArcadeDB gRpcW (com.arcadedb:arcadedb-grpcw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-grpcw/)
     (Apache 2) ArcadeDB Integration (com.arcadedb:arcadedb-integration:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-integration/)
     (Apache 2) ArcadeDB Metrics (com.arcadedb:arcadedb-metrics:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-metrics/)
     (Apache 2) ArcadeDB MongoDB Wire Protocol (com.arcadedb:arcadedb-mongodbw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-mongodbw/)
     (Apache 2) ArcadeDB Network (com.arcadedb:arcadedb-network:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-network/)
     (Apache 2) ArcadeDB PostgresW (com.arcadedb:arcadedb-postgresw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-postgresw/)
     (Apache 2) ArcadeDB RedisW (com.arcadedb:arcadedb-redisw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-redisw/)
     (Apache 2) ArcadeDB Server (com.arcadedb:arcadedb-server:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-server/)
     (Apache 2) ArcadeDB Studio (com.arcadedb:arcadedb-studio:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-studio/)
     (Apache 2) ArcadeDB Test Utils (com.arcadedb:arcadedb-test-utils:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-test-utils/)
     (Apache License 2.0) HPPC Collections (com.carrotsearch:hppc:0.7.1 - http://labs.carrotsearch.com/hppc.html/hppc)
     (Apache License 2.0) Metrics Core (com.codahale.metrics:metrics-core:3.0.2 - http://metrics.codahale.com/metrics-core/)
     (The Apache License, Version 2.0) com.conversantmedia:disruptor (com.conversantmedia:disruptor:1.2.21 - https://github.com/conversant/disruptor)
     (Apache License 2.0) Jackson-annotations (com.fasterxml.jackson.core:jackson-annotations:2.20 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) Jackson-annotations (com.fasterxml.jackson.core:jackson-annotations:2.21 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) Jackson-core (com.fasterxml.jackson.core:jackson-core:2.21.1 - https://github.com/FasterXML/jackson-core)
     (Apache License 2.0) Jackson-core (com.fasterxml.jackson.core:jackson-core:2.21.2 - https://github.com/FasterXML/jackson-core)
     (Apache License 2.0) jackson-databind (com.fasterxml.jackson.core:jackson-databind:2.21.1 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) jackson-databind (com.fasterxml.jackson.core:jackson-databind:2.21.2 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) Jackson-dataformat-YAML (com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.21.1 - https://github.com/FasterXML/jackson-dataformats-text)
     (Apache License 2.0) Jackson datatype: JSR310 (com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.21.1 - https://github.com/FasterXML/jackson-modules-java8/jackson-datatype-jsr310)
     (Apache License 2.0) Caffeine cache (com.github.ben-manes.caffeine:caffeine:2.3.1 - https://github.com/ben-manes/caffeine)
     (Apache License 2.0) docker-java-api (com.github.docker-java:docker-java-api:3.7.1 - https://github.com/docker-java/docker-java)
     (Apache License 2.0) docker-java-transport (com.github.docker-java:docker-java-transport:3.7.1 - https://github.com/docker-java/docker-java)
     (Apache License 2.0) docker-java-transport-zerodep (com.github.docker-java:docker-java-transport-zerodep:3.7.1 - https://github.com/docker-java/docker-java)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) btf (com.github.java-json-tools:btf:1.3 - https://github.com/java-json-tools/btf)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) jackson-coreutils (com.github.java-json-tools:jackson-coreutils:2.0 - https://github.com/java-json-tools/jackson-coreutils)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) jackson-coreutils-equivalence (com.github.java-json-tools:jackson-coreutils-equivalence:1.0 - https://github.com/java-json-tools/jackson-coreutils)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) json-patch (com.github.java-json-tools:json-patch:1.13 - https://github.com/java-json-tools/json-patch)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) json-schema-core (com.github.java-json-tools:json-schema-core:1.2.14 - https://github.com/java-json-tools/json-schema-core)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) json-schema-validator (com.github.java-json-tools:json-schema-validator:2.2.14 - https://github.com/java-json-tools/json-schema-validator)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) msg-simple (com.github.java-json-tools:msg-simple:1.2 - https://github.com/java-json-tools/msg-simple)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) uri-template (com.github.java-json-tools:uri-template:0.10 - https://github.com/java-json-tools/uri-template)
     (Apache License 2.0) (GNU Lesser General Public License) javaparser-core (com.github.javaparser:javaparser-core:3.26.3 - https://github.com/javaparser/javaparser-core)
     (Apache License 2.0) JCIP Annotations under Apache License (com.github.stephenc.jcip:jcip-annotations:1.0-1 - http://stephenc.github.com/jcip-annotations)
     (Apache License 2.0) Google Android Annotations Library (com.google.android:annotations:4.1.1.4 - http://source.android.com/)
     (BSD 3-Clause) API Common (com.google.api:api-common:2.53.0 - https://github.com/googleapis/sdk-platform-java)
     (Apache License 2.0) proto-google-common-protos (com.google.api.grpc:proto-google-common-protos:2.67.0 - https://github.com/googleapis/sdk-platform-java)

@github-actions
Copy link
Copy Markdown
Contributor

github-actions bot commented Apr 5, 2026

📜 License Compliance Check

✅ License check passed. See artifacts for full report.

License Summary (first 50 lines)

Lists of 400 third-party dependencies.
     (Apache License 2.0) LZ4 Java Compression (at.yawk.lz4:lz4-java:1.10.4 - https://github.com/yawkat/lz4-java)
     (EPL 2.0) (GNU Lesser General Public License) Logback Classic Module (ch.qos.logback:logback-classic:1.5.32 - http://logback.qos.ch/logback-classic)
     (EPL 2.0) (GNU Lesser General Public License) Logback Core Module (ch.qos.logback:logback-core:1.5.32 - http://logback.qos.ch/logback-core)
     (Apache 2) ArcadeDB BOLT Protocol (com.arcadedb:arcadedb-bolt:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-bolt/)
     (Apache 2) ArcadeDB Console (com.arcadedb:arcadedb-console:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-console/)
     (Apache 2) ArcadeDB Engine (com.arcadedb:arcadedb-engine:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-engine/)
     (Apache 2) ArcadeDB GraphQL (com.arcadedb:arcadedb-graphql:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-graphql/)
     (Apache 2) ArcadeDB Gremlin (com.arcadedb:arcadedb-gremlin:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-gremlin/)
     (Apache 2) ArcadeDB gRPC Stubs (com.arcadedb:arcadedb-grpc:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-grpc/)
     (Apache 2) ArcadeDB gRPC Client (com.arcadedb:arcadedb-grpc-client:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-grpc-client/)
     (Apache 2) ArcadeDB gRpcW (com.arcadedb:arcadedb-grpcw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-grpcw/)
     (Apache 2) ArcadeDB Integration (com.arcadedb:arcadedb-integration:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-integration/)
     (Apache 2) ArcadeDB Metrics (com.arcadedb:arcadedb-metrics:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-metrics/)
     (Apache 2) ArcadeDB MongoDB Wire Protocol (com.arcadedb:arcadedb-mongodbw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-mongodbw/)
     (Apache 2) ArcadeDB Network (com.arcadedb:arcadedb-network:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-network/)
     (Apache 2) ArcadeDB PostgresW (com.arcadedb:arcadedb-postgresw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-postgresw/)
     (Apache 2) ArcadeDB RedisW (com.arcadedb:arcadedb-redisw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-redisw/)
     (Apache 2) ArcadeDB Server (com.arcadedb:arcadedb-server:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-server/)
     (Apache 2) ArcadeDB Studio (com.arcadedb:arcadedb-studio:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-studio/)
     (Apache 2) ArcadeDB Test Utils (com.arcadedb:arcadedb-test-utils:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-test-utils/)
     (Apache License 2.0) HPPC Collections (com.carrotsearch:hppc:0.7.1 - http://labs.carrotsearch.com/hppc.html/hppc)
     (Apache License 2.0) Metrics Core (com.codahale.metrics:metrics-core:3.0.2 - http://metrics.codahale.com/metrics-core/)
     (The Apache License, Version 2.0) com.conversantmedia:disruptor (com.conversantmedia:disruptor:1.2.21 - https://github.com/conversant/disruptor)
     (Apache License 2.0) Jackson-annotations (com.fasterxml.jackson.core:jackson-annotations:2.20 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) Jackson-annotations (com.fasterxml.jackson.core:jackson-annotations:2.21 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) Jackson-core (com.fasterxml.jackson.core:jackson-core:2.21.1 - https://github.com/FasterXML/jackson-core)
     (Apache License 2.0) Jackson-core (com.fasterxml.jackson.core:jackson-core:2.21.2 - https://github.com/FasterXML/jackson-core)
     (Apache License 2.0) jackson-databind (com.fasterxml.jackson.core:jackson-databind:2.21.1 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) jackson-databind (com.fasterxml.jackson.core:jackson-databind:2.21.2 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) Jackson-dataformat-YAML (com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.21.1 - https://github.com/FasterXML/jackson-dataformats-text)
     (Apache License 2.0) Jackson datatype: JSR310 (com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.21.1 - https://github.com/FasterXML/jackson-modules-java8/jackson-datatype-jsr310)
     (Apache License 2.0) Caffeine cache (com.github.ben-manes.caffeine:caffeine:2.3.1 - https://github.com/ben-manes/caffeine)
     (Apache License 2.0) docker-java-api (com.github.docker-java:docker-java-api:3.7.1 - https://github.com/docker-java/docker-java)
     (Apache License 2.0) docker-java-transport (com.github.docker-java:docker-java-transport:3.7.1 - https://github.com/docker-java/docker-java)
     (Apache License 2.0) docker-java-transport-zerodep (com.github.docker-java:docker-java-transport-zerodep:3.7.1 - https://github.com/docker-java/docker-java)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) btf (com.github.java-json-tools:btf:1.3 - https://github.com/java-json-tools/btf)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) jackson-coreutils (com.github.java-json-tools:jackson-coreutils:2.0 - https://github.com/java-json-tools/jackson-coreutils)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) jackson-coreutils-equivalence (com.github.java-json-tools:jackson-coreutils-equivalence:1.0 - https://github.com/java-json-tools/jackson-coreutils)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) json-patch (com.github.java-json-tools:json-patch:1.13 - https://github.com/java-json-tools/json-patch)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) json-schema-core (com.github.java-json-tools:json-schema-core:1.2.14 - https://github.com/java-json-tools/json-schema-core)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) json-schema-validator (com.github.java-json-tools:json-schema-validator:2.2.14 - https://github.com/java-json-tools/json-schema-validator)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) msg-simple (com.github.java-json-tools:msg-simple:1.2 - https://github.com/java-json-tools/msg-simple)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) uri-template (com.github.java-json-tools:uri-template:0.10 - https://github.com/java-json-tools/uri-template)
     (Apache License 2.0) (GNU Lesser General Public License) javaparser-core (com.github.javaparser:javaparser-core:3.26.3 - https://github.com/javaparser/javaparser-core)
     (Apache License 2.0) JCIP Annotations under Apache License (com.github.stephenc.jcip:jcip-annotations:1.0-1 - http://stephenc.github.com/jcip-annotations)
     (Apache License 2.0) Google Android Annotations Library (com.google.android:annotations:4.1.1.4 - http://source.android.com/)
     (BSD 3-Clause) API Common (com.google.api:api-common:2.53.0 - https://github.com/googleapis/sdk-platform-java)
     (Apache License 2.0) proto-google-common-protos (com.google.api.grpc:proto-google-common-protos:2.67.0 - https://github.com/googleapis/sdk-platform-java)

@lvca
Copy link
Copy Markdown
Member Author

lvca commented Apr 5, 2026

Created first HA benchmark with multiple server, testing an insert and the results are VERY PPOR:

==========================================================================================
  ArcadeDB HA Insert Benchmark
==========================================================================================
  Records: 5,000 (warmup: 500)  |  Batch size: 100  |  Vertex type: Sensor
==========================================================================================

  Scenario                                          Throughput    Avg       P99       Max
  ------------------------------------------------  ----------  --------  --------  --------
  1 server (no HA) - embedded                       1,546 op/s    647 us   4,362 us   4,362 us
  3 servers (HA)   - embedded on leader                53 op/s 18,831 us  28,638 us  28,638 us
  5 servers (HA)   - embedded on leader                49 op/s 20,399 us  24,416 us  24,416 us
  3 servers (HA)   - remote via follower proxy         59 op/s 16,861 us  23,483 us  38,443 us
  3 servers (HA)   - concurrent (3 threads)            77 op/s 39,060 us  52,003 us 106,720 us
  5 servers (HA)   - concurrent (5 threads)            67 op/s 74,731 us 157,715 us 270,166 us

Key findings:

  • HA overhead: ~29x slower than no-HA (1,546 vs 53 ops/sec per batch of 100). Each batch must replicate via gRPC to majority before committing. This is expected for synchronous Raft consensus.
  • 3 vs 5 servers: Minimal difference (53 vs 49 ops/sec). MAJORITY quorum means 3 servers need 2 acks, 5 servers need 3 acks - only one extra network round-trip.
  • Follower proxy: Actually slightly faster than direct-to-leader embedded (59 vs 53 ops/sec) because single-insert commands don't batch and Ratis can pipeline better with small payloads.
  • Concurrent throughput: 3 threads achieve 77 ops/sec total (vs 59 single-thread) - ~30% improvement from parallelism. 5 threads get 67 ops/sec, likely bottlenecked on the single leader processing all writes.
  • P99 latency: Scales with concurrency (23ms single -> 52ms 3-thread -> 157ms 5-thread) due to leader contention.

The benchmark is tagged @tag("benchmark") so it's excluded from normal test runs. Run with: mvn test -pl server -Dtest=HAInsertBenchmark -Dgroups=benchmark

We need to analyze this before merging.

@claude
Copy link
Copy Markdown

claude bot commented Apr 5, 2026

Code review posted - see full review details in the next comment

@claude
Copy link
Copy Markdown

claude bot commented Apr 5, 2026

test4 ignore

@claude
Copy link
Copy Markdown

claude bot commented Apr 5, 2026

Deleting test - please ignore this comment

@claude
Copy link
Copy Markdown

claude bot commented Apr 5, 2026

Code Review - Replace custom Raft protocol with Apache Ratis

This is a major and well-structured change. Replacing a custom, ad-hoc Raft implementation with Apache Ratis is the right call for long-term reliability. The overall architecture is sound, the test coverage is impressive (36 tests + Docker e2e), and the documentation is thorough. Below are issues found during review.


CRITICAL

1. System.out.println in production code (HALog.java)

HALog.log() unconditionally calls System.out.println in addition to LogManager. Per project conventions (CLAUDE.md): remove any System.out you used for debug when you have finished. This will spam stdout in production when arcadedb.ha.logVerbose is nonzero.

Also, the .replace in the format call is a no-op, and building a String.format() string just for the println while also calling LogManager is redundant. The println should simply be removed.


2. Platform-default charset in UUID.nameUUIDFromBytes(clusterName.getBytes()) in RaftHAServer.java

All nodes must agree on the same groupId or they will never form a cluster. Using .getBytes() without a charset argument is platform-dependent. This should use StandardCharsets.UTF_8 explicitly, as already done for the token derivation two lines later:

UUID.nameUUIDFromBytes(clusterName.getBytes(StandardCharsets.UTF_8))

HIGH

3. Busy-wait polling in waitForAppliedIndex() and waitForLocalApply()

Both methods spin with Thread.sleep(10) up to the quorum timeout. This adds up to 10ms latency on every READ_YOUR_WRITES and LINEARIZABLE read and holds a thread. Since ArcadeDBStateMachine already has lastAppliedIndex as an AtomicLong, a LockSupport-based condition or a CompletableFuture completed when the index advances would be lower-latency.


4. Dead code: forwardCommandToLeader() private method in ReplicatedDatabase.java

The method is defined but never called. All command paths now throw ServerIsNotTheLeaderException instead, relying on the HTTP proxy. The method should be removed to avoid confusion.


5. No connection/read timeout on the HTTP proxy in AbstractServerHttpHandler.proxyToLeader()

The call to new java.net.URI(targetUrl).toURL().openConnection() has no setConnectTimeout or setReadTimeout. If the leader is slow or unreachable, the follower thread blocks indefinitely. Explicit timeouts matching quorumTimeout should be set.


6. Fully qualified names in production code

CLAUDE.md says do not use fully qualified names if possible, always import the class. There are many FQN usages in production code in RaftHAServer.java and ReplicatedDatabase.java. For example: peerHttpAddresses is declared as new java.util.concurrent.ConcurrentHashMap, lagMonitorExecutor as java.util.concurrent.ScheduledExecutorService, and getFollowerStates returns java.util.List of java.util.Map. These should be imported at the top of the file.


MEDIUM

7. isCurrentNodeLeader() race in ArcadeDBStateMachine.applyTransaction()

The state machine skips apply on the leader (returns early if isCurrentNodeLeader()). During a leadership transition, a node that was leader when commit2ndPhase() ran could lose leadership before applyTransaction() is called, causing double-application of WAL page changes. Worth adding a comment explaining why this is safe in the Ratis model, or verifying it actually cannot happen.


8. Breaking change: connectCluster / disconnectCluster throw UnsupportedOperationException

Existing users calling POST /api/v1/server with command connect or disconnect will get a 500 response instead of a helpful message. These should return a proper HTTP 400 with a migration hint rather than an uncaught exception.


9. Spurious double-block in exception handler in AbstractServerHttpHandler.java

There is an extra unnecessary block wrapping the proxyToLeader try block in the ServerIsNotTheLeaderException catch handler. It is a no-op but visually confusing and should be removed.


LOW / NITS

10. Cluster token written back to GlobalConfiguration

initClusterToken() calls configuration.setValue(GlobalConfiguration.HA_CLUSTER_TOKEN, this.clusterToken). Since GlobalConfiguration values can appear in server info dumps and logs, the derived token (which is a function of the root password) could be inadvertently exposed. Consider keeping it only in the clusterToken field without writing it back to the global config.

11. getFollowerStates() index correlation assumption

The method assumes matchIndices[i] corresponds to the i-th non-self peer in raftGroup.getPeers(). This assumption depends on Ratis internal ordering that may not be guaranteed across versions. Should be validated against the Ratis API contract.

12. System.out.println in test infrastructure

RaftClusterStarter System.out usage is fine as CLI output, but actual test classes such as RaftHAComprehensiveIT should not emit to stdout to avoid cluttering CI output.


Summary

Category Count
Critical 2
High 4
Medium 3
Low/Nit 3

The architectural direction is excellent and the test suite is thorough. The System.out.println in HALog and the platform-default charset in the group ID derivation are the two that should be addressed before merge.

@codecov
Copy link
Copy Markdown

codecov bot commented Apr 5, 2026

Codecov Report

❌ Patch coverage is 45.34510% with 1362 lines in your changes missing coverage. Please review.
✅ Project coverage is 64.22%. Comparing base (2421e69) to head (2cacd59).
⚠️ Report is 3 commits behind head on main.

Files with missing lines Patch % Lines
...ava/com/arcadedb/server/ha/ratis/RaftHAServer.java 50.18% 347 Missing and 54 partials ⚠️
...arcadedb/server/ha/ratis/ArcadeDBStateMachine.java 43.12% 246 Missing and 27 partials ⚠️
.../server/http/handler/PostServerCommandHandler.java 1.60% 175 Missing and 9 partials ⚠️
...ava/com/arcadedb/server/ha/ratis/RaftLogEntry.java 42.46% 118 Missing and 8 partials ⚠️
.../arcadedb/server/ha/ratis/SnapshotHttpHandler.java 2.54% 115 Missing ⚠️
...m/arcadedb/server/ha/ratis/RaftGroupCommitter.java 42.50% 57 Missing and 12 partials ⚠️
...server/http/handler/AbstractServerHttpHandler.java 55.45% 30 Missing and 19 partials ⚠️
...ava/com/arcadedb/server/ha/ReplicatedDatabase.java 71.87% 27 Missing and 9 partials ⚠️
.../main/java/com/arcadedb/server/ArcadeDBServer.java 45.65% 24 Missing and 1 partial ⚠️
.../java/com/arcadedb/remote/RemoteHttpComponent.java 48.14% 9 Missing and 5 partials ⚠️
... and 13 more
Additional details and impacted files
@@            Coverage Diff             @@
##             main    #3798      +/-   ##
==========================================
- Coverage   64.88%   64.22%   -0.66%     
==========================================
  Files        1581     1557      -24     
  Lines      116573   116457     -116     
  Branches    24722    24797      +75     
==========================================
- Hits        75636    74799     -837     
- Misses      30593    31336     +743     
+ Partials    10344    10322      -22     

☔ 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.
  • 📦 JS Bundle Analysis: Save yourself from yourself by tracking and limiting bundle sizes in JS merges.

@github-actions
Copy link
Copy Markdown
Contributor

github-actions bot commented Apr 5, 2026

📜 License Compliance Check

✅ License check passed. See artifacts for full report.

License Summary (first 50 lines)

Lists of 400 third-party dependencies.
     (Apache License 2.0) LZ4 Java Compression (at.yawk.lz4:lz4-java:1.10.4 - https://github.com/yawkat/lz4-java)
     (EPL 2.0) (GNU Lesser General Public License) Logback Classic Module (ch.qos.logback:logback-classic:1.5.32 - http://logback.qos.ch/logback-classic)
     (EPL 2.0) (GNU Lesser General Public License) Logback Core Module (ch.qos.logback:logback-core:1.5.32 - http://logback.qos.ch/logback-core)
     (Apache 2) ArcadeDB BOLT Protocol (com.arcadedb:arcadedb-bolt:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-bolt/)
     (Apache 2) ArcadeDB Console (com.arcadedb:arcadedb-console:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-console/)
     (Apache 2) ArcadeDB Engine (com.arcadedb:arcadedb-engine:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-engine/)
     (Apache 2) ArcadeDB GraphQL (com.arcadedb:arcadedb-graphql:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-graphql/)
     (Apache 2) ArcadeDB Gremlin (com.arcadedb:arcadedb-gremlin:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-gremlin/)
     (Apache 2) ArcadeDB gRPC Stubs (com.arcadedb:arcadedb-grpc:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-grpc/)
     (Apache 2) ArcadeDB gRPC Client (com.arcadedb:arcadedb-grpc-client:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-grpc-client/)
     (Apache 2) ArcadeDB gRpcW (com.arcadedb:arcadedb-grpcw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-grpcw/)
     (Apache 2) ArcadeDB Integration (com.arcadedb:arcadedb-integration:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-integration/)
     (Apache 2) ArcadeDB Metrics (com.arcadedb:arcadedb-metrics:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-metrics/)
     (Apache 2) ArcadeDB MongoDB Wire Protocol (com.arcadedb:arcadedb-mongodbw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-mongodbw/)
     (Apache 2) ArcadeDB Network (com.arcadedb:arcadedb-network:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-network/)
     (Apache 2) ArcadeDB PostgresW (com.arcadedb:arcadedb-postgresw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-postgresw/)
     (Apache 2) ArcadeDB RedisW (com.arcadedb:arcadedb-redisw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-redisw/)
     (Apache 2) ArcadeDB Server (com.arcadedb:arcadedb-server:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-server/)
     (Apache 2) ArcadeDB Studio (com.arcadedb:arcadedb-studio:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-studio/)
     (Apache 2) ArcadeDB Test Utils (com.arcadedb:arcadedb-test-utils:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-test-utils/)
     (Apache License 2.0) HPPC Collections (com.carrotsearch:hppc:0.7.1 - http://labs.carrotsearch.com/hppc.html/hppc)
     (Apache License 2.0) Metrics Core (com.codahale.metrics:metrics-core:3.0.2 - http://metrics.codahale.com/metrics-core/)
     (The Apache License, Version 2.0) com.conversantmedia:disruptor (com.conversantmedia:disruptor:1.2.21 - https://github.com/conversant/disruptor)
     (Apache License 2.0) Jackson-annotations (com.fasterxml.jackson.core:jackson-annotations:2.20 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) Jackson-annotations (com.fasterxml.jackson.core:jackson-annotations:2.21 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) Jackson-core (com.fasterxml.jackson.core:jackson-core:2.21.1 - https://github.com/FasterXML/jackson-core)
     (Apache License 2.0) Jackson-core (com.fasterxml.jackson.core:jackson-core:2.21.2 - https://github.com/FasterXML/jackson-core)
     (Apache License 2.0) jackson-databind (com.fasterxml.jackson.core:jackson-databind:2.21.1 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) jackson-databind (com.fasterxml.jackson.core:jackson-databind:2.21.2 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) Jackson-dataformat-YAML (com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.21.1 - https://github.com/FasterXML/jackson-dataformats-text)
     (Apache License 2.0) Jackson datatype: JSR310 (com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.21.1 - https://github.com/FasterXML/jackson-modules-java8/jackson-datatype-jsr310)
     (Apache License 2.0) Caffeine cache (com.github.ben-manes.caffeine:caffeine:2.3.1 - https://github.com/ben-manes/caffeine)
     (Apache License 2.0) docker-java-api (com.github.docker-java:docker-java-api:3.7.1 - https://github.com/docker-java/docker-java)
     (Apache License 2.0) docker-java-transport (com.github.docker-java:docker-java-transport:3.7.1 - https://github.com/docker-java/docker-java)
     (Apache License 2.0) docker-java-transport-zerodep (com.github.docker-java:docker-java-transport-zerodep:3.7.1 - https://github.com/docker-java/docker-java)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) btf (com.github.java-json-tools:btf:1.3 - https://github.com/java-json-tools/btf)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) jackson-coreutils (com.github.java-json-tools:jackson-coreutils:2.0 - https://github.com/java-json-tools/jackson-coreutils)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) jackson-coreutils-equivalence (com.github.java-json-tools:jackson-coreutils-equivalence:1.0 - https://github.com/java-json-tools/jackson-coreutils)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) json-patch (com.github.java-json-tools:json-patch:1.13 - https://github.com/java-json-tools/json-patch)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) json-schema-core (com.github.java-json-tools:json-schema-core:1.2.14 - https://github.com/java-json-tools/json-schema-core)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) json-schema-validator (com.github.java-json-tools:json-schema-validator:2.2.14 - https://github.com/java-json-tools/json-schema-validator)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) msg-simple (com.github.java-json-tools:msg-simple:1.2 - https://github.com/java-json-tools/msg-simple)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) uri-template (com.github.java-json-tools:uri-template:0.10 - https://github.com/java-json-tools/uri-template)
     (Apache License 2.0) (GNU Lesser General Public License) javaparser-core (com.github.javaparser:javaparser-core:3.26.3 - https://github.com/javaparser/javaparser-core)
     (Apache License 2.0) JCIP Annotations under Apache License (com.github.stephenc.jcip:jcip-annotations:1.0-1 - http://stephenc.github.com/jcip-annotations)
     (Apache License 2.0) Google Android Annotations Library (com.google.android:annotations:4.1.1.4 - http://source.android.com/)
     (BSD 3-Clause) API Common (com.google.api:api-common:2.53.0 - https://github.com/googleapis/sdk-platform-java)
     (Apache License 2.0) proto-google-common-protos (com.google.api.grpc:proto-google-common-protos:2.67.0 - https://github.com/googleapis/sdk-platform-java)

@claude
Copy link
Copy Markdown

claude bot commented Apr 5, 2026

Code Review - PR 3798: Replace Custom Raft with Apache Ratis

This is a substantial, well-motivated architectural replacement. Replacing a hand-rolled Raft protocol with Apache Ratis is the right call - the PR description is honest about the limitations of the old implementation. The new code is well-structured and comes with an impressive test suite (17 comprehensive tests + Docker e2e tests with network fault injection). The documentation in docs/arcadedb-ha-26.4.1.md is thorough.

Below are the issues found, grouped by severity.


SECURITY - High

1. Weak cluster token derivation (RaftHAServer.initClusterToken())

UUID.nameUUIDFromBytes uses MD5 (RFC 4122 v3) - not a cryptographic primitive. Given the cluster name is often visible via /api/v1/server, an attacker can enumerate common passwords offline and derive the token. The cluster token then bypasses per-request authentication in validateClusterForwardedAuth().

Recommendation: use HMAC-SHA256 (javax.crypto.Mac) which is in the JDK with no new dependencies:

Mac mac = Mac.getInstance("HmacSHA256");
mac.init(new SecretKeySpec(rootPassword.getBytes(StandardCharsets.UTF_8), "HmacSHA256"));
this.clusterToken = Base64.getUrlEncoder().withoutPadding().encodeToString(mac.doFinal(clusterName.getBytes(StandardCharsets.UTF_8)));

2. SnapshotHttpHandler - unauthenticated database download surface

The snapshot endpoint is registered on basicRoutes, bypassing the standard AbstractServerHttpHandler security pipeline. It only checks Basic auth and silently returns HTTP 401 for any other credential type. Any node with valid user credentials can download the full raw database. This is a significant data exfiltration surface. The endpoint should at minimum require the root role and ideally also accept the cluster token for inter-node use.


SECURITY - Medium

3. validateClusterForwardedAuth() - username blindly trusted from header

After validating the cluster token, the username from X-ArcadeDB-Forwarded-User is accepted without further verification. If the cluster token is compromised (per issue 1), an attacker can impersonate any user including root.

4. proxyToLeader() - silent privilege escalation to root

conn.setRequestProperty(HEADER_CLUSTER_TOKEN, raftHA.getClusterToken());
conn.setRequestProperty(HEADER_FORWARDED_USER, "root");

If a request arrives at a follower with no Authorization header (from a handler where isRequireAuthentication() returns false), the proxy silently escalates to root. This is unintended privilege escalation.

5. Snapshot downloaded over plain HTTP with no integrity verification

installDatabaseSnapshot() uses "http://" with no HTTPS and no hash/signature verification of the downloaded ZIP. A MITM can serve a malicious database during follower catch-up. The zip-slip protection is correct but does not protect at the data level. Should be in Known Limitations at minimum.


BUGS - High

6. System.out.println in production HALog.java

HALog.java is in src/main/java and contains a live System.out.println. This will print to stdout in every production deployment. CLAUDE.md explicitly prohibits System.out in finished code.

Additionally the format string mutation message.replace("%d", "%s") will corrupt format strings that contain %d as part of a literal string rather than a format specifier.

7. DDL replication inside the database write lock (ReplicatedDatabase.recordFileChanges())

replicateTransaction is called inside executeInWriteLock, which means it blocks for the full Raft round-trip (up to quorumTimeout ms) while holding the database-level write lock. This blocks all reads and writes on that database for the entire replication latency. The old code explicitly sent the command outside the exclusive lock. This regression can cause severe throughput degradation for DDL operations.


BUGS - Medium

8. Files.list() stream not closed in RaftHAServer.startService()

final boolean storageExists = java.nio.file.Files.exists(storagePath)
    && java.nio.file.Files.list(storagePath).findAny().isPresent();

Files.list() wraps a directory file descriptor and must be closed via try-with-resources. Leaving it open leaks a file descriptor.

9. Potential double-apply for TRANSACTION_FORWARD entries

applyTransactionEntry() correctly skips WAL apply on the leader to avoid double-application. applyTransactionForwardEntry() has no equivalent guard. If the TRANSACTION_FORWARD path is used, the leader applies the WAL twice - a data corruption risk.

10. getFollowerStates() - array index alignment not guaranteed

The code assumes getFollowerMatchIndices() / getFollowerNextIndices() arrays align with getPeers() (excluding self) in iteration order. Ratis does not explicitly document this ordering guarantee. If misaligned, replication lag metrics in Studio will be attributed to the wrong peers.


PERFORMANCE

11. proxyToLeader() uses HttpURLConnection with no connection pool or read timeout

Every proxied follower-to-leader request opens a new TCP connection. The default read timeout is infinite (0). Under load this creates many TIME_WAIT sockets and risks blocking proxy threads indefinitely.

12. Busy-poll in waitForLocalApply() under LINEARIZABLE reads

The 10ms poll loop for applied index creates unnecessary scheduling overhead. Ratis provides RaftClient.io().watch() for notification-based waiting.


CODE QUALITY

13. Fully-qualified class names in production code

Multiple files use java.util.List, java.nio.file.Files, java.net.HttpURLConnection etc. inline instead of imports. CLAUDE.md: "don't use fully qualified names if possible, always import the class and just use the name."

14. Duplicate Javadoc block on waitForLocalApply()

Two consecutive /** */ blocks on the same method - the first is orphaned.

15. forwardCommandToLeader() in ReplicatedDatabase appears to be dead code

Writes now throw ServerIsNotTheLeaderException with HTTP proxy forwarding. This method should be removed or clearly documented.

16. Magic byte 'C' for state machine routing

ASCII 'C' = 67 is used as a routing discriminator without being defined as a constant or integrated into the EntryType enum (which already defines byte codes 1 and 2). This would silently collide with any future entry type byte value of 67.


Summary

Severity Count
Security - High 2
Security - Medium 3
Bug - High 2
Bug - Medium 3
Performance 2
Code Quality 4

Must fix before merge:

  • Issue 1: Cluster token derivation (MD5/UUID -> HMAC-SHA256)
  • Issue 6: System.out.println in HALog.java
  • Issue 7: DDL replication inside the database write lock
  • Issue 8: Files.list() stream leak

The overall design and test coverage are excellent. The Ratis integration is architecturally sound. These are fixable issues that should be addressed before merging to main.

@claude
Copy link
Copy Markdown

claude bot commented Apr 9, 2026

Test pseudo-code block:

lifecycleExecutor.submit(() -> {
  try {
    installDatabasesFromLeader();
  } catch (Exception e) {
    needsSnapshotDownload.set(true);
  }
});

@claude
Copy link
Copy Markdown

claude bot commented Apr 9, 2026

Positive Observations

  • readBoundedString() in RaftLogEntry is well-defended: validates declared length before allocation and checks against remaining buffer bytes - good protection against corrupted or malicious entries.
  • SnapshotHttpHandler correctly sanitizes the database name for Content-Disposition (prevents header injection) and validates for path traversal characters.
  • constantTimeTokenEquals() uses hash comparison to prevent timing attacks on the cluster token.
  • ClusterTokenAuthIT covers the rejection paths thoroughly (wrong token, missing user header, unknown user).
  • leaveCluster() correctly transfers leadership before self-removal - easy to get wrong, handled properly here.
  • The semaphore in SnapshotHttpHandler is always released in a finally block.
  • The e2e Docker tests covering network partition, rolling restart, and Toxiproxy-based packet loss/latency injection are an excellent addition for validating real failure scenarios.

Overall this is production-quality work. Items 1 and 2 are worth addressing before merge, particularly the stuck-follower case after snapshot download failure.

@claude
Copy link
Copy Markdown

claude bot commented Apr 9, 2026

Minor Issues

4. arcadedb.ha.replicationLagWarning units are ambiguous

The description says "Raft log gap threshold for lag warnings (0=disabled)" but the unit is Raft log indices, not milliseconds. Operators familiar with typical monitoring thresholds will likely misread this. Suggest appending: "A value of N means the replica is N committed log entries behind the leader."

5. RaftLogEntry hardcoded limits - lines 138-140

MAX_DELTA_SIZE = 1_000_000 limits WAL delta entries per transaction. Large index rebuilds or bulk loads could exceed this. Consider exposing it via GlobalConfiguration, or at minimum document what happens when it is exceeded (currently an IllegalArgumentException during deserialization, which surfaces as a Raft apply failure).

6. Missing input validation for ha add peer

The command parses peerId and address from user input without validating host:port format or port bounds. Ratis produces opaque errors for invalid addresses; an early validation check would give operators a cleaner error message.

7. ha leave with exactly 2 nodes

leaveCluster() correctly guards livePeers.size() <= 1, but in a 2-node cluster removing one node leaves a 1-node cluster unable to elect a new leader if the remaining node also fails. Worth documenting in the HA guide that a 2-node cluster provides no fault tolerance.

@claude
Copy link
Copy Markdown

claude bot commented Apr 9, 2026

Client-visible Behavior Worth Documenting

3. Double-timeout in RaftGroupCommitter - line 84-100

When a transaction is dispatched to Raft but the initial timeoutMs expires, the code correctly waits an additional getQuorumTimeout() to prevent phantom commits (entry replicated on followers but commit2ndPhase never called on leader). This is the right design, but the effective maximum client-visible latency becomes timeoutMs + quorumTimeout, not just timeoutMs. The arcadedb.ha.quorumTimeout description should mention this so operators do not treat timeoutMs as a hard upper bound.

@claude
Copy link
Copy Markdown

claude bot commented Apr 9, 2026

2. TOCTOU race in SnapshotHttpHandler - line 138 vs 166

existsDatabase() is checked at line 138, but getDatabase() at line 166 is called only after exchange.startBlocking() and response headers are written. If the database is dropped in between, server.getDatabase(databaseName) returns null and line 168 (db.getEmbedded()) throws NPE. A null guard before unwrapping is sufficient.

@claude
Copy link
Copy Markdown

claude bot commented Apr 9, 2026

PR Review: Replace custom Raft with Apache Ratis

This is a significant, well-architected change. Replacing a custom Raft implementation with Apache Ratis is the right call - Ratis is battle-tested and formally correct. The new code is generally clean and well-structured. Below are observations ranging from correctness issues to minor suggestions.

Correctness Issues

1. Snapshot download has no automatic retry after full failure - ArcadeDBStateMachine.java line 853-863

If installDatabasesFromLeader() exhausts all its internal retries and throws, the catch block only logs the error. The needsSnapshotDownload flag was already set to false via compareAndSet before the submit, so there is no automatic retry. A follower in this state is stuck with stale or missing databases until the next leader-change event triggers notifyLeaderChanged() again - which may never happen in a stable cluster.

Consider resetting the flag on failure so the next leader notification triggers a fresh attempt. In the catch block, add needsSnapshotDownload.set(true) before or after the existing log call. The corrected structure:

lifecycleExecutor.submit(() -> {
  try {
    installDatabasesFromLeader();
  } catch (final Exception e) {
    needsSnapshotDownload.set(true);
    LogManager.instance().log(this, Level.SEVERE, e.getMessage());
  }
});

2. TOCTOU race in SnapshotHttpHandler - line 138 vs 166

existsDatabase() is checked at line 138, but getDatabase() at line 166 is called only after exchange.startBlocking() and response headers are written. If the database is dropped in between, server.getDatabase(databaseName) returns null and line 168 (db.getEmbedded()) throws NPE. A null guard before unwrapping is sufficient.

Client-visible Behavior Worth Documenting

3. Double-timeout in RaftGroupCommitter - line 84-100

When a transaction is dispatched to Raft but the initial timeoutMs expires, the code correctly waits an additional getQuorumTimeout() to prevent phantom commits (entry replicated on followers but commit2ndPhase never called on leader). This is the right design, but the effective maximum client-visible latency becomes timeoutMs + quorumTimeout, not just timeoutMs. The arcadedb.ha.quorumTimeout description should mention this so operators do not treat timeoutMs as a hard upper bound.

Minor Issues

4. arcadedb.ha.replicationLagWarning units are ambiguous

The description says "Raft log gap threshold for lag warnings (0=disabled)" but the unit is Raft log indices, not milliseconds. Operators familiar with typical monitoring thresholds will likely misread this. Suggest appending: "A value of N means the replica is N committed log entries behind the leader."

5. RaftLogEntry hardcoded limits - lines 138-140

MAX_DELTA_SIZE = 1_000_000 limits WAL delta entries per transaction. Large index rebuilds or bulk loads could exceed this. Consider exposing it via GlobalConfiguration, or at minimum document what happens when it is exceeded (currently an IllegalArgumentException during deserialization, which surfaces as a Raft apply failure).

6. Missing input validation for ha add peer

The command parses peerId and address from user input without validating host:port format or port bounds. Ratis produces opaque errors for invalid addresses; an early validation check would give operators a cleaner error message.

7. ha leave with exactly 2 nodes

leaveCluster() correctly guards livePeers.size() <= 1, but in a 2-node cluster removing one node leaves a 1-node cluster unable to elect a new leader if the remaining node also fails. Worth documenting in the HA guide that a 2-node cluster provides no fault tolerance.

Positive Observations

  • readBoundedString() in RaftLogEntry is well-defended: validates declared length before allocation and checks against remaining buffer bytes - good protection against corrupted or malicious entries.
  • SnapshotHttpHandler correctly sanitizes the database name for Content-Disposition (prevents header injection) and validates for path traversal characters.
  • constantTimeTokenEquals() uses hash comparison to prevent timing attacks on the cluster token.
  • ClusterTokenAuthIT covers the rejection paths thoroughly (wrong token, missing user header, unknown user).
  • leaveCluster() correctly transfers leadership before self-removal - easy to get wrong, handled properly here.
  • The semaphore in SnapshotHttpHandler is always released in a finally block.
  • The e2e Docker tests covering network partition, rolling restart, and Toxiproxy-based packet loss/latency injection are an excellent addition for validating real failure scenarios.

Overall this is production-quality work. Items 1 and 2 are worth addressing before merge, particularly the stuck-follower case after snapshot download failure.

@ArcadeData ArcadeData deleted a comment from claude bot Apr 9, 2026
@ArcadeData ArcadeData deleted a comment from claude bot Apr 9, 2026
@ArcadeData ArcadeData deleted a comment from claude bot Apr 9, 2026
@ArcadeData ArcadeData deleted a comment from claude bot Apr 9, 2026
@ArcadeData ArcadeData deleted a comment from claude bot Apr 9, 2026
@ArcadeData ArcadeData deleted a comment from claude bot Apr 9, 2026
@ArcadeData ArcadeData deleted a comment from claude bot Apr 9, 2026
@ArcadeData ArcadeData deleted a comment from claude bot Apr 9, 2026
@ArcadeData ArcadeData deleted a comment from claude bot Apr 9, 2026
@ArcadeData ArcadeData deleted a comment from claude bot Apr 9, 2026
@claude
Copy link
Copy Markdown

claude bot commented Apr 9, 2026

Code Review - Apache Ratis HA Integration

This is an impressive, well-structured replacement of the custom Raft stack. The core design - Ratis for consensus, group commit for throughput, HTTP-based snapshot transfer, and a clear three-phase commit protocol - is solid. Below are observations grouped by severity.


HIGH - Potential Bugs

RaftGroupCommitter: Misleading exception on queue full

When the submission queue is full, the code throws QuorumNotReachedException. This is the wrong exception - the quorum is fine, the queue is saturated. An operator seeing this exception would investigate cluster health rather than backpressure. Consider a dedicated ReplicationQueueFullException or at minimum include a message that clearly distinguishes the two cases.

ArcadeDBStateMachine: Silent swallow of ConcurrentModificationException

In applyTransactionEntry(), already-applied entries that produce ConcurrentModificationException are caught and silently skipped. This should be logged at ERROR level. Silent failure here could mask data inconsistency that would otherwise surface as a health check failure.

RaftHAServer: Deterministic auto-join jitter

The join retry jitter is derived from peerId.hashCode() - which is deterministic across restarts of the same node. In a StatefulSet scale-up where multiple pods start simultaneously, all pods with the same name compute the same jitter value, creating a thundering herd on the seed node. Use ThreadLocalRandom.current().nextLong(100, 500) for genuine randomness.


MEDIUM - Design and Correctness Concerns

ArcadeDBStateMachine: Hardcoded SNAPSHOT_GAP_TOLERANCE = 10

The threshold at which a follower is forced to download a full snapshot instead of catching up via log replay is hardcoded at 10 entries. For write-heavy workloads this is very aggressive (a brief GC pause could trigger a full snapshot). Expose this as a configuration setting (e.g., arcadedb.ha.snapshotGapTolerance) so operators can tune for their workload.

ArcadeDBStateMachine: No idempotency check in createNewFiles()

The method called during log replay to create new database files does not check whether the file already exists before creating it. On a retry (e.g., leader re-sends after a partial commit), this could overwrite a partially-written file. Add a guard: skip creation if the file already exists and has non-zero size.

ReplicatedDatabase: Schema dirty-flag capture is lossy

In captureReplicationPayload(), the schema dirty flag is checked once. If multiple schema changes happen concurrently between the flag set and the payload capture, only the last snapshot is captured. Consider a generation counter or an explicit change accumulator to make this atomic.

RaftLogEntry: Unknown entry type throws instead of skipping

EntryType.fromCode() throws IllegalArgumentException for unknown type codes. For forward-compatibility (a newer node writing an entry type an older node does not know), this will crash the state machine on the older node during a rolling upgrade. Consider returning Optional.empty() or a UNKNOWN sentinel, and skipping unrecognized entries in applyTransaction() with a warning log.


LOW - Minor / Style

RaftHAServer: Magic numbers should be named constants

Values like 100_000 (PBKDF2 iterations), 10_000 (queue capacity), and the various millisecond timeouts scattered through the file would benefit from named constants with a comment explaining the rationale.

ClusterMonitor: 60-second warning debounce is hardcoded

The lag warning debounce interval is hardcoded at 60 seconds. For operators tuning alerting pipelines it would be useful as a configuration option alongside arcadedb.ha.replicationLagWarning.

SnapshotHttpHandler: MAX_CONCURRENT_SNAPSHOTS = 2 rationale is undocumented

The cap of 2 concurrent snapshot downloads is a reasonable NIC protection measure but unexplained in the code. A comment noting the rationale (and ideally making it configurable) would help future maintainers.


Strengths Worth Noting

  • The three-phase commit protocol in ReplicatedDatabase (capture -> replicate -> local apply) with leader step-down on phase-2 failure is the right safety model.
  • Constant-time cluster token comparison in SnapshotHttpHandler (SHA-256 both sides before comparing) is correct and prevents timing attacks.
  • PBKDF2-derived cluster tokens avoid shipping a static shared secret.
  • The marker-file + atomic directory swap pattern in snapshot installation handles crash-in-the-middle correctly.
  • RaftGroupCommitter's extended-wait logic for already-dispatched entries prevents phantom commit false positives.
  • The read consistency levels (EVENTUAL / READ_YOUR_WRITES / LINEARIZABLE) are well-separated and match documented semantics.
  • The e2e test suite (Toxiproxy for latency/packet-loss, Docker network partitions) provides exactly the right level of confidence for a Raft stack.

Overall this is production-quality work. The main items before merge: fix the misleading exception on queue full, log (not swallow) the ConcurrentModificationException, fix the deterministic jitter, and expose SNAPSHOT_GAP_TOLERANCE as a configuration option.

@codacy-production
Copy link
Copy Markdown

codacy-production bot commented Apr 9, 2026

Not up to standards ⛔

🔴 Issues 1 high · 9 medium · 17 minor

Alerts:
⚠ 27 issues (≤ 0 issues of at least minor severity)

Results:
27 new issues

Category Results
BestPractice 9 medium
ErrorProne 1 high
CodeStyle 17 minor

View in Codacy

🟢 Metrics 834 complexity

Metric Results
Complexity 834

View in Codacy

TIP This summary will be updated as you push new changes. Give us feedback

@github-actions
Copy link
Copy Markdown
Contributor

github-actions bot commented Apr 9, 2026

📜 License Compliance Check

✅ License check passed. See artifacts for full report.

License Summary (first 50 lines)

Lists of 384 third-party dependencies.
     (Apache License 2.0) LZ4 Java Compression (at.yawk.lz4:lz4-java:1.10.4 - https://github.com/yawkat/lz4-java)
     (EPL 2.0) (GNU Lesser General Public License) Logback Classic Module (ch.qos.logback:logback-classic:1.5.32 - http://logback.qos.ch/logback-classic)
     (EPL 2.0) (GNU Lesser General Public License) Logback Core Module (ch.qos.logback:logback-core:1.5.32 - http://logback.qos.ch/logback-core)
     (Apache 2) ArcadeDB BOLT Protocol (com.arcadedb:arcadedb-bolt:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-bolt/)
     (Apache 2) ArcadeDB Console (com.arcadedb:arcadedb-console:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-console/)
     (Apache 2) ArcadeDB Engine (com.arcadedb:arcadedb-engine:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-engine/)
     (Apache 2) ArcadeDB GraphQL (com.arcadedb:arcadedb-graphql:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-graphql/)
     (Apache 2) ArcadeDB Gremlin (com.arcadedb:arcadedb-gremlin:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-gremlin/)
     (Apache 2) ArcadeDB gRPC Stubs (com.arcadedb:arcadedb-grpc:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-grpc/)
     (Apache 2) ArcadeDB gRPC Client (com.arcadedb:arcadedb-grpc-client:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-grpc-client/)
     (Apache 2) ArcadeDB gRpcW (com.arcadedb:arcadedb-grpcw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-grpcw/)
     (Apache 2) ArcadeDB Integration (com.arcadedb:arcadedb-integration:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-integration/)
     (Apache 2) ArcadeDB Metrics (com.arcadedb:arcadedb-metrics:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-metrics/)
     (Apache 2) ArcadeDB MongoDB Wire Protocol (com.arcadedb:arcadedb-mongodbw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-mongodbw/)
     (Apache 2) ArcadeDB Network (com.arcadedb:arcadedb-network:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-network/)
     (Apache 2) ArcadeDB PostgresW (com.arcadedb:arcadedb-postgresw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-postgresw/)
     (Apache 2) ArcadeDB RedisW (com.arcadedb:arcadedb-redisw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-redisw/)
     (Apache 2) ArcadeDB Server (com.arcadedb:arcadedb-server:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-server/)
     (Apache 2) ArcadeDB Studio (com.arcadedb:arcadedb-studio:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-studio/)
     (Apache 2) ArcadeDB Test Utils (com.arcadedb:arcadedb-test-utils:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-test-utils/)
     (Apache License 2.0) HPPC Collections (com.carrotsearch:hppc:0.7.1 - http://labs.carrotsearch.com/hppc.html/hppc)
     (Apache License 2.0) Metrics Core (com.codahale.metrics:metrics-core:3.0.2 - http://metrics.codahale.com/metrics-core/)
     (The Apache License, Version 2.0) com.conversantmedia:disruptor (com.conversantmedia:disruptor:1.2.21 - https://github.com/conversant/disruptor)
     (Apache License 2.0) Jackson-annotations (com.fasterxml.jackson.core:jackson-annotations:2.20 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) Jackson-annotations (com.fasterxml.jackson.core:jackson-annotations:2.21 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) Jackson-core (com.fasterxml.jackson.core:jackson-core:2.21.1 - https://github.com/FasterXML/jackson-core)
     (Apache License 2.0) Jackson-core (com.fasterxml.jackson.core:jackson-core:2.21.2 - https://github.com/FasterXML/jackson-core)
     (Apache License 2.0) jackson-databind (com.fasterxml.jackson.core:jackson-databind:2.21.1 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) jackson-databind (com.fasterxml.jackson.core:jackson-databind:2.21.2 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) Jackson-dataformat-YAML (com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.21.1 - https://github.com/FasterXML/jackson-dataformats-text)
     (Apache License 2.0) Jackson datatype: JSR310 (com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.21.1 - https://github.com/FasterXML/jackson-modules-java8/jackson-datatype-jsr310)
     (Apache License 2.0) Caffeine cache (com.github.ben-manes.caffeine:caffeine:2.3.1 - https://github.com/ben-manes/caffeine)
     (Apache License 2.0) docker-java-api (com.github.docker-java:docker-java-api:3.7.1 - https://github.com/docker-java/docker-java)
     (Apache License 2.0) docker-java-transport (com.github.docker-java:docker-java-transport:3.7.1 - https://github.com/docker-java/docker-java)
     (Apache License 2.0) docker-java-transport-zerodep (com.github.docker-java:docker-java-transport-zerodep:3.7.1 - https://github.com/docker-java/docker-java)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) btf (com.github.java-json-tools:btf:1.3 - https://github.com/java-json-tools/btf)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) jackson-coreutils (com.github.java-json-tools:jackson-coreutils:2.0 - https://github.com/java-json-tools/jackson-coreutils)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) jackson-coreutils-equivalence (com.github.java-json-tools:jackson-coreutils-equivalence:1.0 - https://github.com/java-json-tools/jackson-coreutils)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) json-patch (com.github.java-json-tools:json-patch:1.13 - https://github.com/java-json-tools/json-patch)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) json-schema-core (com.github.java-json-tools:json-schema-core:1.2.14 - https://github.com/java-json-tools/json-schema-core)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) json-schema-validator (com.github.java-json-tools:json-schema-validator:2.2.14 - https://github.com/java-json-tools/json-schema-validator)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) msg-simple (com.github.java-json-tools:msg-simple:1.2 - https://github.com/java-json-tools/msg-simple)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) uri-template (com.github.java-json-tools:uri-template:0.10 - https://github.com/java-json-tools/uri-template)
     (Apache License 2.0) (GNU Lesser General Public License) javaparser-core (com.github.javaparser:javaparser-core:3.26.3 - https://github.com/javaparser/javaparser-core)
     (Apache License 2.0) JCIP Annotations under Apache License (com.github.stephenc.jcip:jcip-annotations:1.0-1 - http://stephenc.github.com/jcip-annotations)
     (Apache License 2.0) Google Android Annotations Library (com.google.android:annotations:4.1.1.4 - http://source.android.com/)
     (BSD 3-Clause) API Common (com.google.api:api-common:2.53.0 - https://github.com/googleapis/sdk-platform-java)
     (Apache License 2.0) proto-google-common-protos (com.google.api.grpc:proto-google-common-protos:2.67.0 - https://github.com/googleapis/sdk-platform-java)

@github-actions
Copy link
Copy Markdown
Contributor

github-actions bot commented Apr 9, 2026

📜 License Compliance Check

✅ License check passed. See artifacts for full report.

License Summary (first 50 lines)

Lists of 384 third-party dependencies.
     (Apache License 2.0) LZ4 Java Compression (at.yawk.lz4:lz4-java:1.10.4 - https://github.com/yawkat/lz4-java)
     (EPL 2.0) (GNU Lesser General Public License) Logback Classic Module (ch.qos.logback:logback-classic:1.5.32 - http://logback.qos.ch/logback-classic)
     (EPL 2.0) (GNU Lesser General Public License) Logback Core Module (ch.qos.logback:logback-core:1.5.32 - http://logback.qos.ch/logback-core)
     (Apache 2) ArcadeDB BOLT Protocol (com.arcadedb:arcadedb-bolt:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-bolt/)
     (Apache 2) ArcadeDB Console (com.arcadedb:arcadedb-console:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-console/)
     (Apache 2) ArcadeDB Engine (com.arcadedb:arcadedb-engine:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-engine/)
     (Apache 2) ArcadeDB GraphQL (com.arcadedb:arcadedb-graphql:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-graphql/)
     (Apache 2) ArcadeDB Gremlin (com.arcadedb:arcadedb-gremlin:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-gremlin/)
     (Apache 2) ArcadeDB gRPC Stubs (com.arcadedb:arcadedb-grpc:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-grpc/)
     (Apache 2) ArcadeDB gRPC Client (com.arcadedb:arcadedb-grpc-client:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-grpc-client/)
     (Apache 2) ArcadeDB gRpcW (com.arcadedb:arcadedb-grpcw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-grpcw/)
     (Apache 2) ArcadeDB Integration (com.arcadedb:arcadedb-integration:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-integration/)
     (Apache 2) ArcadeDB Metrics (com.arcadedb:arcadedb-metrics:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-metrics/)
     (Apache 2) ArcadeDB MongoDB Wire Protocol (com.arcadedb:arcadedb-mongodbw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-mongodbw/)
     (Apache 2) ArcadeDB Network (com.arcadedb:arcadedb-network:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-network/)
     (Apache 2) ArcadeDB PostgresW (com.arcadedb:arcadedb-postgresw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-postgresw/)
     (Apache 2) ArcadeDB RedisW (com.arcadedb:arcadedb-redisw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-redisw/)
     (Apache 2) ArcadeDB Server (com.arcadedb:arcadedb-server:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-server/)
     (Apache 2) ArcadeDB Studio (com.arcadedb:arcadedb-studio:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-studio/)
     (Apache 2) ArcadeDB Test Utils (com.arcadedb:arcadedb-test-utils:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-test-utils/)
     (Apache License 2.0) HPPC Collections (com.carrotsearch:hppc:0.7.1 - http://labs.carrotsearch.com/hppc.html/hppc)
     (Apache License 2.0) Metrics Core (com.codahale.metrics:metrics-core:3.0.2 - http://metrics.codahale.com/metrics-core/)
     (The Apache License, Version 2.0) com.conversantmedia:disruptor (com.conversantmedia:disruptor:1.2.21 - https://github.com/conversant/disruptor)
     (Apache License 2.0) Jackson-annotations (com.fasterxml.jackson.core:jackson-annotations:2.20 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) Jackson-annotations (com.fasterxml.jackson.core:jackson-annotations:2.21 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) Jackson-core (com.fasterxml.jackson.core:jackson-core:2.21.1 - https://github.com/FasterXML/jackson-core)
     (Apache License 2.0) Jackson-core (com.fasterxml.jackson.core:jackson-core:2.21.2 - https://github.com/FasterXML/jackson-core)
     (Apache License 2.0) jackson-databind (com.fasterxml.jackson.core:jackson-databind:2.21.1 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) jackson-databind (com.fasterxml.jackson.core:jackson-databind:2.21.2 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) Jackson-dataformat-YAML (com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.21.1 - https://github.com/FasterXML/jackson-dataformats-text)
     (Apache License 2.0) Jackson datatype: JSR310 (com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.21.1 - https://github.com/FasterXML/jackson-modules-java8/jackson-datatype-jsr310)
     (Apache License 2.0) Caffeine cache (com.github.ben-manes.caffeine:caffeine:2.3.1 - https://github.com/ben-manes/caffeine)
     (Apache License 2.0) docker-java-api (com.github.docker-java:docker-java-api:3.7.1 - https://github.com/docker-java/docker-java)
     (Apache License 2.0) docker-java-transport (com.github.docker-java:docker-java-transport:3.7.1 - https://github.com/docker-java/docker-java)
     (Apache License 2.0) docker-java-transport-zerodep (com.github.docker-java:docker-java-transport-zerodep:3.7.1 - https://github.com/docker-java/docker-java)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) btf (com.github.java-json-tools:btf:1.3 - https://github.com/java-json-tools/btf)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) jackson-coreutils (com.github.java-json-tools:jackson-coreutils:2.0 - https://github.com/java-json-tools/jackson-coreutils)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) jackson-coreutils-equivalence (com.github.java-json-tools:jackson-coreutils-equivalence:1.0 - https://github.com/java-json-tools/jackson-coreutils)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) json-patch (com.github.java-json-tools:json-patch:1.13 - https://github.com/java-json-tools/json-patch)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) json-schema-core (com.github.java-json-tools:json-schema-core:1.2.14 - https://github.com/java-json-tools/json-schema-core)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) json-schema-validator (com.github.java-json-tools:json-schema-validator:2.2.14 - https://github.com/java-json-tools/json-schema-validator)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) msg-simple (com.github.java-json-tools:msg-simple:1.2 - https://github.com/java-json-tools/msg-simple)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) uri-template (com.github.java-json-tools:uri-template:0.10 - https://github.com/java-json-tools/uri-template)
     (Apache License 2.0) (GNU Lesser General Public License) javaparser-core (com.github.javaparser:javaparser-core:3.26.3 - https://github.com/javaparser/javaparser-core)
     (Apache License 2.0) JCIP Annotations under Apache License (com.github.stephenc.jcip:jcip-annotations:1.0-1 - http://stephenc.github.com/jcip-annotations)
     (Apache License 2.0) Google Android Annotations Library (com.google.android:annotations:4.1.1.4 - http://source.android.com/)
     (BSD 3-Clause) API Common (com.google.api:api-common:2.53.0 - https://github.com/googleapis/sdk-platform-java)
     (Apache License 2.0) proto-google-common-protos (com.google.api.grpc:proto-google-common-protos:2.67.0 - https://github.com/googleapis/sdk-platform-java)

@claude
Copy link
Copy Markdown

claude bot commented Apr 9, 2026

Code Review - Follow-up

This is a follow-up to earlier Claude review comments. Prior reviews flagged: System.out.println in HALog, missing proxy timeouts, PBKDF2 charset, timing-safe token comparison, busy-wait polling. This comment adds additional observations.


Correction to prior review

ATTRIBUTIONS.md IS updated - The first review incorrectly flagged this as missing. The diff shows +15 additions covering the new Ratis dependencies.


New observations

1. Ratis version discrepancy in documentation

The PR description says Apache Ratis 3.2.1 throughout but server/pom.xml adds version 3.2.2. The docs should reference 3.2.2 consistently.

2. Group commit phantom-commit ambiguity

RaftGroupCommitter marks timed-out entries as cancelled. However, if an entry was already submitted to the Ratis client when the timeout fires, the entry may still be durable in the Raft log even though the local thread received a timeout exception. The client gets an error but the write succeeded. This needs documentation and a test asserting the server does not diverge after such an event.

3. K8s auto-join security

tryAutoJoinCluster discovers peers via DNS. Any pod resolving the expected hostname can join with the cluster token as the only gate. Consider documenting that K8s NetworkPolicy should restrict gRPC port access to only pods in the ArcadeDB StatefulSet.

4. Origin-node skip logic safety

ArcadeDBStateMachine skips WAL application on the origin node. This assumes the leader committed locally in phase 2 before applyTransaction is called. If Ratis replays entries on restart, double-application would corrupt the database. A comment explaining why Ratis will not replay already-applied entries on the origin node would prevent future regressions.

5. Snapshot ZIP path traversal edge case

SnapshotHttpHandler checks for symlinks and path traversal, but ZIP entry names come from actual filenames. If a database filename itself contains .., the ZIP entry could embed a traversal path. Verify ZIP entry names are normalized via Paths.get(filename).getFileName().toString() when added to the stream.

6. connectCluster / disconnectCluster breaking change

These throw UnsupportedOperationException, giving migrating users a confusing 500. An HTTP 400 with 'Use ha add peer/ha remove peer instead' would improve the migration experience.

7. TODO section in docs

docs/arcadedb-ha-26.4.1.md contains a TODO section with planned features. Remove it or convert to GitHub issues before merging.


Priority summary

  • Critical: System.out.println in HALog.java (prior review)
  • High: Missing timeout on proxyToLeader (prior review)
  • High: Non-constant-time cluster token comparison (prior review)
  • Medium: Phantom-commit/double-apply ambiguity (items 2, 4)
  • Medium: connectCluster/disconnectCluster breaking change (item 6)

The overall architecture is solid and the test coverage is impressive. The Docker e2e tests covering network partitions and rolling restarts are especially valuable for a consensus system.

@github-actions
Copy link
Copy Markdown
Contributor

📜 License Compliance Check

✅ License check passed. See artifacts for full report.

License Summary (first 50 lines)

Lists of 384 third-party dependencies.
     (Apache License 2.0) LZ4 Java Compression (at.yawk.lz4:lz4-java:1.10.4 - https://github.com/yawkat/lz4-java)
     (EPL 2.0) (GNU Lesser General Public License) Logback Classic Module (ch.qos.logback:logback-classic:1.5.32 - http://logback.qos.ch/logback-classic)
     (EPL 2.0) (GNU Lesser General Public License) Logback Core Module (ch.qos.logback:logback-core:1.5.32 - http://logback.qos.ch/logback-core)
     (Apache 2) ArcadeDB BOLT Protocol (com.arcadedb:arcadedb-bolt:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-bolt/)
     (Apache 2) ArcadeDB Console (com.arcadedb:arcadedb-console:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-console/)
     (Apache 2) ArcadeDB Engine (com.arcadedb:arcadedb-engine:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-engine/)
     (Apache 2) ArcadeDB GraphQL (com.arcadedb:arcadedb-graphql:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-graphql/)
     (Apache 2) ArcadeDB Gremlin (com.arcadedb:arcadedb-gremlin:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-gremlin/)
     (Apache 2) ArcadeDB gRPC Stubs (com.arcadedb:arcadedb-grpc:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-grpc/)
     (Apache 2) ArcadeDB gRPC Client (com.arcadedb:arcadedb-grpc-client:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-grpc-client/)
     (Apache 2) ArcadeDB gRpcW (com.arcadedb:arcadedb-grpcw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-grpcw/)
     (Apache 2) ArcadeDB Integration (com.arcadedb:arcadedb-integration:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-integration/)
     (Apache 2) ArcadeDB Metrics (com.arcadedb:arcadedb-metrics:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-metrics/)
     (Apache 2) ArcadeDB MongoDB Wire Protocol (com.arcadedb:arcadedb-mongodbw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-mongodbw/)
     (Apache 2) ArcadeDB Network (com.arcadedb:arcadedb-network:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-network/)
     (Apache 2) ArcadeDB PostgresW (com.arcadedb:arcadedb-postgresw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-postgresw/)
     (Apache 2) ArcadeDB RedisW (com.arcadedb:arcadedb-redisw:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-redisw/)
     (Apache 2) ArcadeDB Server (com.arcadedb:arcadedb-server:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-server/)
     (Apache 2) ArcadeDB Studio (com.arcadedb:arcadedb-studio:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-studio/)
     (Apache 2) ArcadeDB Test Utils (com.arcadedb:arcadedb-test-utils:26.4.1-SNAPSHOT - https://arcadedata.com/arcadedb-test-utils/)
     (Apache License 2.0) HPPC Collections (com.carrotsearch:hppc:0.7.1 - http://labs.carrotsearch.com/hppc.html/hppc)
     (Apache License 2.0) Metrics Core (com.codahale.metrics:metrics-core:3.0.2 - http://metrics.codahale.com/metrics-core/)
     (The Apache License, Version 2.0) com.conversantmedia:disruptor (com.conversantmedia:disruptor:1.2.21 - https://github.com/conversant/disruptor)
     (Apache License 2.0) Jackson-annotations (com.fasterxml.jackson.core:jackson-annotations:2.20 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) Jackson-annotations (com.fasterxml.jackson.core:jackson-annotations:2.21 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) Jackson-core (com.fasterxml.jackson.core:jackson-core:2.21.1 - https://github.com/FasterXML/jackson-core)
     (Apache License 2.0) Jackson-core (com.fasterxml.jackson.core:jackson-core:2.21.2 - https://github.com/FasterXML/jackson-core)
     (Apache License 2.0) jackson-databind (com.fasterxml.jackson.core:jackson-databind:2.21.1 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) jackson-databind (com.fasterxml.jackson.core:jackson-databind:2.21.2 - https://github.com/FasterXML/jackson)
     (Apache License 2.0) Jackson-dataformat-YAML (com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.21.1 - https://github.com/FasterXML/jackson-dataformats-text)
     (Apache License 2.0) Jackson datatype: JSR310 (com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.21.1 - https://github.com/FasterXML/jackson-modules-java8/jackson-datatype-jsr310)
     (Apache License 2.0) Caffeine cache (com.github.ben-manes.caffeine:caffeine:2.3.1 - https://github.com/ben-manes/caffeine)
     (Apache License 2.0) docker-java-api (com.github.docker-java:docker-java-api:3.7.1 - https://github.com/docker-java/docker-java)
     (Apache License 2.0) docker-java-transport (com.github.docker-java:docker-java-transport:3.7.1 - https://github.com/docker-java/docker-java)
     (Apache License 2.0) docker-java-transport-zerodep (com.github.docker-java:docker-java-transport-zerodep:3.7.1 - https://github.com/docker-java/docker-java)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) btf (com.github.java-json-tools:btf:1.3 - https://github.com/java-json-tools/btf)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) jackson-coreutils (com.github.java-json-tools:jackson-coreutils:2.0 - https://github.com/java-json-tools/jackson-coreutils)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) jackson-coreutils-equivalence (com.github.java-json-tools:jackson-coreutils-equivalence:1.0 - https://github.com/java-json-tools/jackson-coreutils)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) json-patch (com.github.java-json-tools:json-patch:1.13 - https://github.com/java-json-tools/json-patch)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) json-schema-core (com.github.java-json-tools:json-schema-core:1.2.14 - https://github.com/java-json-tools/json-schema-core)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) json-schema-validator (com.github.java-json-tools:json-schema-validator:2.2.14 - https://github.com/java-json-tools/json-schema-validator)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) msg-simple (com.github.java-json-tools:msg-simple:1.2 - https://github.com/java-json-tools/msg-simple)
     (Apache Software License, version 2.0) (Lesser General Public License, version 3 or greater) uri-template (com.github.java-json-tools:uri-template:0.10 - https://github.com/java-json-tools/uri-template)
     (Apache License 2.0) (GNU Lesser General Public License) javaparser-core (com.github.javaparser:javaparser-core:3.26.3 - https://github.com/javaparser/javaparser-core)
     (Apache License 2.0) JCIP Annotations under Apache License (com.github.stephenc.jcip:jcip-annotations:1.0-1 - http://stephenc.github.com/jcip-annotations)
     (Apache License 2.0) Google Android Annotations Library (com.google.android:annotations:4.1.1.4 - http://source.android.com/)
     (BSD 3-Clause) API Common (com.google.api:api-common:2.53.0 - https://github.com/googleapis/sdk-platform-java)
     (Apache License 2.0) proto-google-common-protos (com.google.api.grpc:proto-google-common-protos:2.67.0 - https://github.com/googleapis/sdk-platform-java)

@claude
Copy link
Copy Markdown

claude bot commented Apr 10, 2026

PR Review: Replace custom Raft protocol with Apache Ratis

This is a major, well-scoped architectural improvement. Replacing the hand-rolled Raft implementation with Apache Ratis 3.2.2 is the right call - battle-tested consensus via a library used by Apache Ozone and IoTDB is strictly better than maintaining a custom protocol. The breadth of test coverage (unit, integration, Docker e2e, network-partition, packet-loss) is excellent and gives confidence in the correctness of the new implementation.

Below are findings grouped by severity.


Security

[Concern] Weak cluster token when root password is empty

In RaftHAServer.initClusterToken(), the PBKDF2 derivation uses the root password as the secret input. If arcadedb.ha.clusterToken is not explicitly set and the root password is empty (common in dev/test or when auth is disabled), the derived token is effectively predictable from the cluster name alone. The existing warning log may not reach operators who don't monitor logs closely.

Suggestion: In addition to logging the warning, refuse to start HA mode with an auto-derived token when the root password is empty and no explicit token is configured - or at minimum escalate to an ERROR-level log entry (not just a WARNING).

[Minor] Plain HTTP for inter-node snapshot download

SnapshotHttpHandler serves database snapshots (WAL data) over plain HTTP. This is acknowledged in the code with comments pointing to VPN/mTLS deployment requirements, but it should be explicitly documented in both the server configuration docs and the new docs/arcadedb-ha-26.4.1.md as a mandatory network-level security requirement for production, not just a recommendation.

[Positive] Constant-time token comparison

Using SHA-256 + MessageDigest.isEqual() in both SnapshotHttpHandler and AbstractServerHttpHandler for cluster token validation correctly prevents timing-based side-channel attacks. Good practice.

[Positive] Path traversal prevention in snapshot handler

The database name validation in SnapshotHttpHandler (rejecting /, \, .., null bytes) is correct. Also good: symlinks in the database directory are excluded from the ZIP.


Correctness

[Minor] RaftGroupCommitter queue back-pressure behavior

submitAndWait() throws when MAX_QUEUE_SIZE (10,000) is reached. Under sustained overload, callers will see a ReplicationQueueFullException which becomes a transaction failure from the client's perspective. This is acceptable behavior, but the Javadoc on submitAndWait and the queue constant itself would benefit from a comment clarifying the expected operational response (tune write rate, scale cluster, etc.) so operators know what to do when they see this error in logs.

[Minor] restartRatisIfNeeded() restart loop timing

The health monitor checks every 3 seconds and calls restartRatisIfNeeded() if the server is CLOSED or CLOSING. If the underlying cause of the Ratis server closing is a persistent infrastructure issue (e.g., port conflict, bad storage path), this will loop indefinitely. Consider adding a consecutive-failure counter with exponential backoff and an eventual FATAL-level alert after N restart attempts.

[Positive] Origin-node skip logic

The applyTransactionEntry logic that skips WAL replay on the originating node uses the immutable originPeerId from the log entry (not the current role) - this is the correct way to handle leadership changes between commit and apply, and the detailed comment explaining the safety argument is appreciated.

[Positive] Schema change ordering

Files are created before WAL apply and removed after - correct ordering that prevents orphan pages or missing targets during follower replay.


Performance

[Positive] Group commit via RaftGroupCommitter

Batching multiple pending Raft entries into a single pipelined async submission is a sound optimization that amortizes gRPC round-trip latency under concurrent write load. The atomic state machine (STATE_PENDING / STATE_DISPATCHED / STATE_CANCELLED) is clean and avoids TOCTOU races between timeout cancellations and dispatch.

[Minor] Snapshot ZIP created in-memory before sending

SnapshotHttpHandler accumulates the entire ZIP into a ByteArrayOutputStream before writing to the HTTP response. For large databases this will cause a significant heap spike. Consider streaming directly to the response OutputStream (e.g., ZipOutputStream wrapping exchange.getResponseBody()) with Transfer-Encoding: chunked. The concurrent download semaphore already bounds the number of simultaneous snapshots, which helps, but the per-download memory cost remains.

[Minor] HALog.getLevel() refreshes GlobalConfiguration every 5 seconds

This is a reasonable trade-off for runtime log-level changes without restart. No issue here, just worth noting it adds a System.currentTimeMillis() call on every log-path invocation - negligible in practice.


Code Style / Minor Issues

[Minor] HA_REPLICATION_LAG_WARNING default of 1000

The description says "number of uncommitted entries" - this is a Raft log index gap, not wall-clock milliseconds - but the configuration key name (replicationLagWarning) and the description in the settings table (uses the word "lag") may lead operators to think it's milliseconds. Consider renaming the setting to arcadedb.ha.replicationLagEntries or updating the description to explicitly say "log index gap (number of entries, not milliseconds)".

[Positive] No System.out.println, printStackTrace, or debug artifacts

All logging goes through the ArcadeDB logger or HALog, which is correct.

[Positive] ATTRIBUTIONS.md and NOTICE updated

Apache Ratis (Apache 2.0) is properly attributed. Required NOTICE content is incorporated. License compliance is clean.


Test Coverage

The test suite is comprehensive:

  • RaftLogEntryTest - serialization round-trips (good unit test)
  • RaftHAServerIT, RaftReplicationIT, RaftHAComprehensiveIT - cluster lifecycle and data consistency
  • ReadConsistencyIT - all three consistency levels covered
  • ClusterTokenAuthIT - auth validation
  • Docker e2e tests with Toxiproxy for network chaos - this is exactly the right level of testing for a consensus protocol

One gap worth noting: there are no tests for the "weak token on empty root password" scenario described above.


Summary

This is a high-quality replacement of a complex internal system. The architecture is sound, the security practices are solid for the threat model (trusted private network), and the test coverage is exceptional for a PR of this size. The items above are mostly hardening suggestions rather than blocking issues. The snapshot streaming concern is the most operationally impactful one for large datasets.

Great work to @lvca and @robfrank.

@claude
Copy link
Copy Markdown

claude bot commented Apr 10, 2026

Code Review - Apache Ratis HA Integration

Overall this is a well-engineered, high-quality PR that replaces a custom Raft implementation with battle-tested Apache Ratis 3.2.2. The architecture is clean, the concurrency handling is careful, and the test coverage is comprehensive.

Positive Observations

  • Clean architecture: Good separation between RaftHAServer (lifecycle), ArcadeDBStateMachine (replication), and RaftGroupCommitter (batching)
  • Smart batching: RaftGroupCommitter with drainTo() and configurable batch size amortizes gRPC round-trips effectively
  • Concurrency: Consistent use of AtomicInteger/AtomicLong/AtomicBoolean in hot paths, ReentrantLock+Condition for waiter signaling in waitForAppliedIndex()
  • Security: PBKDF2-HMAC-SHA256 with 100k iterations for cluster token; path traversal defense in SnapshotHttpHandler; sanitized database name in header injection prevention
  • Forward compatibility: Unrecognized entry types return null rather than throwing, enabling rolling upgrades
  • Test coverage: Split-brain, rolling restart, network partition, and consistency level tests are all present

Issues

Critical / Should Fix Before Merge

1. Snapshot gap detection race - ArcadeDBStateMachine.java

needsSnapshotDownload.set(true) is set in reinitialize(), but the actual download is deferred to notifyLeaderChanged(). If a follower restarts with a snapshot gap and no leader change event fires (e.g., leader is stable), the gap is never resolved - the node is permanently stale without any error surfacing.

Suggested fix: In the state machine, if needsSnapshotDownload is still true after a timeout from reinitialize(), trigger a background download task rather than waiting indefinitely for a leader change notification.

2. leaderReady = true set in catch block - RaftHAServer.java

leaderReady = true is set in the catch block of waitForLocalApply() in addition to the success path. If catch-up fails, this violates the invariant that the leader should not serve linearizable reads during catch-up failures, potentially allowing stale reads.

Suggested fix: Only set leaderReady = true on the success path.

3. Leadership transfer notification race - RaftHAServer.java

The leadership transfer loop waits on leaderChangeNotifier.notifyAll(), but if the Ratis event fires before wait() is entered, the notification is missed and the thread sleeps for the full 5-second deadline unnecessarily.

Suggested fix: Use a volatile flag checked before wait(), or replace the pattern with a CountDownLatch.

4. Log entry serialization format has no version byte - RaftLogEntry.java

Entry type codes (1, 3) are hardcoded with no format version. During rolling upgrades, if the serialization format changes, followers with the new code receiving old-format entries (or vice versa) may silently deserialize corrupted data.

Suggested fix: Add a version byte after the entry type code and validate at deserialization time.


High - Should Address Soon

5. Negative replication lag possible - ClusterMonitor.java

leaderCommitIndex - matchIndex is computed without a lower bound. If matchIndex ever exceeds leaderCommitIndex (e.g., due to a Ratis implementation detail or a bug), the Studio dashboard would display a negative lag value and lag alerts could fire incorrectly.

Suggested fix: Math.max(0, leaderCommitIndex - matchIndex)

6. Actual worst-case quorum timeout can exceed documented 2x - RaftHAServer.java / RaftGroupCommitter.java

The Javadoc states effective timeout is "up to 2x quorumTimeout", but because the second wait in submitAndWait() uses a fresh haServer.getQuorumTimeout() rather than the remaining deadline, network delays can push the total to ~3x. Operators may misconfigure timeouts based on the documented 2x expectation.

Suggested fix: Compute extendedTimeout = min(remainingDeadline, quorumTimeout) at the second wait, not a full fresh timeout.


Medium / Low

7. Empty cluster token silently degrades auth - RaftHAServer.java

The PBKDF2 derivation accepts whatever is in arcadedb.ha.clusterToken including an empty string. An empty token would not fail at startup but would make inter-node auth trivially bypassable.

Suggested fix: Validate non-empty at startup and log a warning if shorter than 32 characters.

8. Health monitor detects stalled server but does not recover - RaftHAServer.java

The health monitor loop logs when the server is CLOSED but takes no recovery action. The surrounding code comment implies automatic recovery; the Javadoc should clarify that manual intervention is needed.

9. Snapshot completion not logged at INFO - SnapshotHttpHandler.java

Snapshot start errors are logged at SEVERE; successful completion has no corresponding INFO log, making operational tracing asymmetric.


Minor Style Notes

  • A few places use fully-qualified class names inline rather than imports (CLAUDE.md discourages this)
  • Some debug LogManager.getLogger() calls could use the class-level logger for consistency

The batching, lock-free state tracking, and origin-node skip logic in the state machine are all well done. Issues #1-4 are worth addressing before merge given this is consensus/replication code where subtle races can manifest only under production load.

@claude
Copy link
Copy Markdown

claude bot commented Apr 10, 2026

Code Review: Replace custom Raft protocol with Apache Ratis

This is a substantial and well-motivated architectural change. Replacing a home-grown Raft-like implementation with Apache Ratis (battle-tested in Apache Ozone, IoTDB, and Alluxio) is sound engineering. The PR has clearly gone through several review iterations with many issues already resolved. The test coverage - unit, integration, and Docker e2e - is thorough.

Below are the remaining issues I found, grouped by severity.


Critical - Should Fix Before Merge

1. Snapshot gap never resolved if leader does not change (ArcadeDBStateMachine.java)

needsSnapshotDownload.set(true) is set in reinitialize(), but the actual download is triggered in notifyLeaderChanged(). If a follower restarts with a snapshot gap but the leader is stable and no leadership change fires, the follower is permanently stale with no error surfaced to operators and no automatic recovery.

Fix: After reinitialize() sets the flag, start a background watchdog that triggers snapshot download directly if the flag is still set after a configurable timeout, rather than waiting indefinitely for a leadership change.

2. leaderReady = true set in the catch block (RaftHAServer.java)

In the leader catch-up path, leaderReady = true appears to be set on both the success path and in the exception catch block. If catch-up fails, the leader is marked ready and immediately begins serving LINEARIZABLE reads with a stale applied index, breaking the LINEARIZABLE guarantee.

Fix: Only set leaderReady = true on the explicit success path, not in the catch block.

3. Lost-wakeup race in leadership transfer wait (RaftHAServer.java)

The leadership transfer loop checks a condition and then calls leaderChangeNotifier.wait(). If the Ratis event fires between the condition check and the wait(), the notification is lost and the caller sleeps for the full 5-second deadline unnecessarily. This is a classic lost-wakeup race.

Fix: Replace with a CountDownLatch or use a volatile boolean checked inside the synchronized block before calling wait().

4. No serialization format version in RaftLogEntry (RaftLogEntry.java)

Entry type codes are hardcoded with no format version byte. During rolling upgrades, old and new nodes may disagree on deserialization, silently producing corrupted data rather than a clear error.

Fix: Add a 1-byte version field after the entry type byte. Validate at deserialization and fail fast on unrecognized versions.


High - Should Address Soon

5. Snapshot ZIP fully buffered in heap (SnapshotHttpHandler.java)

The entire database ZIP is accumulated into a ByteArrayOutputStream before streaming to the HTTP response. For large databases, this causes a massive heap spike per snapshot request. The concurrent download semaphore limits parallelism but not the per-download memory cost.

Fix: Stream the ZipOutputStream directly to exchange.getOutputStream(). Undertow supports chunked transfer encoding natively.

6. Effective submitAndWait() timeout is ~3x quorumTimeout, not 2x (RaftGroupCommitter.java)

The Javadoc documents the effective timeout as "up to 2x quorumTimeout". However, the second wait creates a fresh deadline rather than using the remaining time from the first wait. Under network delay, operators who dimension their system based on the documented 2x will be surprised by the actual behavior.

Fix: Compute remainingDeadline = firstDeadline - System.currentTimeMillis() and use min(remainingDeadline, quorumTimeout) for the second wait.

7. Negative replication lag possible (ClusterMonitor.java)

leaderCommitIndex - matchIndex has no lower-bound guard. If Ratis internal state temporarily causes matchIndex > leaderCommitIndex, the Studio dashboard shows negative lag and the warning threshold comparison misfires.

Fix: Math.max(0, leaderCommitIndex - matchIndex).

8. setConfiguration() return value ignored in K8s auto-join (RaftHAServer.java)

tempClient.admin().setConfiguration(newPeers) returns a RaftClientReply indicating success or failure. The return value is discarded and the log always says "successfully joined". If the operation fails (e.g., leader election in progress), the node incorrectly believes it joined the cluster.

Fix: Check reply.isSuccess() and throw or retry on failure.

9. restartRatisIfNeeded() loops indefinitely on persistent failure (RaftHAServer.java)

If the cause of the Ratis server closing is persistent (port conflict, bad storage path, full disk), the health monitor calls restart() every 3 seconds forever, generating noise without alerting operators.

Fix: Add a consecutive-failure counter with exponential backoff. After N failures, log at SEVERE and stop retrying (require manual intervention).


Security

10. Weak cluster token when root password is empty (RaftHAServer.java)

initClusterToken() uses PBKDF2-HMAC-SHA256 with the root password as the secret. If the root password is empty (common in dev/CI or when auth is not configured), the derived token is fully predictable from the cluster name alone. A warning is logged, but operators who don't monitor logs will silently run with broken auth.

Fix: Refuse to start HA mode with an auto-derived token when rootPassword is null or empty and no explicit arcadedb.ha.clusterToken is configured. Throw an IllegalStateException or require an explicit token.

11. getBytes() without charset in token comparison (AbstractServerHttpHandler.java)

MessageDigest.isEqual(expectedToken.getBytes(), providedToken.getBytes()) uses the platform-default charset. The tokens are hex-encoded ASCII, so this is unlikely to cause an actual failure, but it violates the project convention of always specifying charset.

Fix: Use StandardCharsets.UTF_8 in both places.

12. Snapshot transferred over plain HTTP (ArcadeDBStateMachine.java)

installDatabaseSnapshot() hardcodes "http://". Full WAL snapshots are transferred in cleartext between cluster nodes. The documentation should explicitly mark this as a known limitation requiring VPN or mTLS rather than just a recommendation.


Code Quality (per CLAUDE.md conventions)

13. arcadedb.ha.replicationLagWarning name is misleading

The setting measures a Raft log index gap (number of entries), not time. The word "lag" will mislead operators into thinking it is milliseconds. Consider renaming to arcadedb.ha.replicationLagEntries or explicitly stating "log index gap (entries, not milliseconds)" in all descriptions and Studio UI labels.


What Was Already Fixed (well done)

  • System.out.println in HALog.java - removed
  • Files.list() stream not closed - fixed with try-with-resources
  • clusterName.getBytes() without charset - fixed to StandardCharsets.UTF_8
  • SnapshotHttpHandler path parameter extraction - fixed
  • Dead forwardCommandToLeader() method - removed
  • PBKDF2-HMAC-SHA256 with 100k iterations for token derivation (replaced MD5-based approach)
  • MessageDigest.isEqual() for constant-time token comparison
  • Missing proxy timeouts in proxyToLeader()
  • ThreadLocal READ_CONSISTENCY_CONTEXT leak - fixed
  • walFile.delete() return value - addressed
  • ATTRIBUTIONS.md and NOTICE updated for Apache Ratis - confirmed

Summary

# Issue Severity
1 Snapshot gap never resolved without leader change Critical
2 leaderReady = true set in failure catch block Critical
3 Lost-wakeup race in leadership transfer Critical
4 No format version in RaftLogEntry serialization Critical
5 Snapshot ZIP fully buffered in heap High
6 submitAndWait() timeout documented incorrectly (2x vs 3x) High
7 Negative replication lag possible in ClusterMonitor High
8 setConfiguration() return value ignored in K8s auto-join High
9 Health monitor retries indefinitely on persistent failure High
10 Empty root password produces predictable cluster token Security
11 getBytes() without charset in token comparison Low
12 Snapshot over plain HTTP - needs explicit documentation Security/Docs
13 replicationLagWarning name misleads on unit (entries vs ms) Quality

The PR is architecturally solid and the test coverage is impressive. Issues 1-4 should be addressed before merge. Issues 5-9 can be tracked as follow-up. The rest are polish.

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