Skip to content

Migrate BookkeeperProtocol from protobuf-java to LightProto#4780

Merged
merlimat merged 12 commits into
masterfrom
lightproto-migration-bkproto
May 8, 2026
Merged

Migrate BookkeeperProtocol from protobuf-java to LightProto#4780
merlimat merged 12 commits into
masterfrom
lightproto-migration-bkproto

Conversation

@merlimat
Copy link
Copy Markdown
Contributor

@merlimat merlimat commented May 7, 2026

Summary

Migrates the BookkeeperProtocol.proto wire protocol to use LightProto for serialization. Combined with the prior migration of DataFormats / DbLedgerStorageDataFormats (#4779), this drops the protobuf-java runtime dependency from bookkeeper-proto entirely.

LightProto produces wire-compatible output with protoc for the same .proto, so on-the-wire bookie/client compatibility is preserved.

Lifecycle notes

  • LightProto messages parsed from a ByteBuf hold lazy references into that buffer for field access. The decoders now call materialize() on parsed Request / Response / AuthMessage instances so they survive after the source buffer is released.
  • Server response paths that put entry payloads into ReadLacResponse or ReadResponse now copy the bytes via ByteBufUtil.getBytes(...), matching the previous ByteString.copyFrom semantics.

Drive-by fix

processWriteLacRequestV3 / processReadLacRequestV3 in BookieRequestProcessor were ordering work on r.getAddRequest().getLedgerId() instead of the matching WriteLac/ReadLac request. With protobuf this returned a default 0 for the unset field; with LightProto it throws IllegalStateException. Now uses the correct accessor.

Test plan

  • BookieProtoEncodingTest, TestBookieRequestProcessor, TestPerChannelBookieClient, WriteEntryProcessorV3Test, LongPollReadEntryProcessorV3Test, GetBookieInfoProcessorV3Test, ForceLedgerProcessorV3Test
  • TestBackwardCompatCMS42 (V2 wire protocol with AuthMessage)
  • BookKeeperTest, BookieClientTest, BookieInitializationTest, DbLedgerStorageTest, CookieTest, TestLedgerMetadataSerDe, TestLedgerUnderreplicationManager, TestGetBookieInfoTimeout
  • All 169 tests pass

merlimat added 6 commits May 7, 2026 16:27
The batchReadUnconfirmedAsync method added in #4739 calls LOG.error(...),
but LedgerHandle was migrated to slog and only has a lowercase `log`
field. Master fails to compile.

Convert the call to the slog builder style used elsewhere in the file.
… to LightProto

Replace Google's protobuf-java with StreamNative LightProto for the storage and
metadata formats in `bookkeeper-proto`. The wire protocol (`BookkeeperProtocol`)
remains on protobuf-java for now.

LightProto generates mutable, reusable, ByteBuf-aware classes with built-in
proto2 TextFormat (de)serialization (via `generateTextFormat=true`), so the
existing TextFormat-based znode payloads (cookies, auditor votes, lock data,
underreplication entries, layout) round-trip byte-identically.

Notable behavior changes:
- `BookKeeper.DigestType.toProtoDigestType` now returns the LightProto-generated
  enum (same constants, different package).
- v3 ledger metadata uses a hand-rolled length-prefixed delimited writer/reader
  matching protobuf's `writeDelimitedTo`/`mergeDelimitedFrom`.
The existing exclude `~org.apache.bookkeeper.proto.DataFormats.*` matched
protobuf's nested `DataFormats$LedgerMetadataFormat` etc. LightProto generates
the same messages as flat top-level classes (`LedgerMetadataFormat` directly
in `org.apache.bookkeeper.proto`), so those weren't excluded and triggered
12 bugs in the generated code (bit-twiddling, exposed internal byte arrays,
etc.) that aren't actionable.

Replace the obsolete `DataFormats.*` exclude with explicit per-message
patterns covering both packages and `LightProtoCodec` (also generated
per-package).
Migrates the BookkeeperProtocol.proto wire protocol to use LightProto for
serialization. Combined with the prior migrations of DataFormats and
DbLedgerStorageDataFormats, this drops the protobuf-java runtime dependency
from bookkeeper-proto entirely.

LightProto produces wire-compatible output with protoc for the same .proto,
so on-the-wire bookie/client compatibility is preserved.

Notes on lifecycle handling:
- LightProto messages parsed from a ByteBuf hold lazy references into that
  buffer for field access. The decoders now call materialize() on parsed
  Request/Response/AuthMessage instances so they survive after the source
  buffer is released.
- Server response paths that put entry payloads into ReadLacResponse or
  ReadResponse now copy the bytes via ByteBufUtil.getBytes(...), matching
  the previous ByteString.copyFrom semantics.

Drive-by fix: processWriteLacRequestV3/processReadLacRequestV3 were
ordering work on r.getAddRequest().getLedgerId() instead of the matching
WriteLac/ReadLac request. With protobuf this returned a default 0 for the
unset field; with LightProto it throws IllegalStateException.
@merlimat merlimat force-pushed the lightproto-migration-bkproto branch from 15303d5 to 2222dc6 Compare May 7, 2026 23:36
protobuf-java is no longer pulled in transitively by bookkeeper-server,
so the shaded jars no longer contain (shaded) protobuf classes, and the
flat lightproto-generated classes have replaced the BookkeeperProtocol
outer class.

- BookKeeperServerShadedJarTest / DistributedLogCoreShadedJarTest:
  drop the now-irrelevant testProtobufShadedPath checks and switch the
  BookkeeperProtocol presence check to a real lightproto class
  (AddRequest).
- Drop the dead com.google.protobuf:protobuf-java <include> from the
  three shade plugin configs (bookkeeper-server-shaded,
  bookkeeper-server-tests-shaded, distributedlog-core-shaded).
@merlimat merlimat requested a review from lhotari May 8, 2026 01:22
Copy link
Copy Markdown
Member

@lhotari lhotari left a comment

Choose a reason for hiding this comment

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

Some comments regarding ByteBufList handling and passing a ByteBuf instance directly (instead of calling ByteBufUtil.getBytes(bytebuf))

lac = requestProcessor.bookie.getExplicitLac(ledgerId);
if (lac != null) {
readLacResponse.setLacBody(ByteString.copyFrom(lac.nioBuffer()));
readLacResponse.setLacBody(ByteBufUtil.getBytes(lac));
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Pass ByteBuf directly? Also ensure whether .retain() also needs to be called.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I wouldn't try to improve performance as part of this change, since the retention itself can be quite tricky to track.

LedgerData newLedgerData = LedgerData.newBuilder(ledgerData)
.setExplicitLac(ByteString.copyFrom(lac.nioBuffer())).build();
LedgerData newLedgerData = new LedgerData().copyFrom(ledgerData)
.setExplicitLac(ByteBufUtil.getBytes(lac));
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Pass ByteBuf directly?
Special attention is needed for the ref count in this case since ByteString.copyFrom is used in the previous code. Does .retain() need to be called?

} else {
try {
readResponseBuilder.setBody(ByteString.copyFrom(entryBody.nioBuffer()));
readResponseBuilder.setBody(ByteBufUtil.getBytes(entryBody));
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Pass ByteBuf directly?
Special attention is needed for the ref count in this case since ByteString.copyFrom is used in the previous code. Does .retain() need to be called?

} else {
status = StatusCode.EOK;
readResponse.setBody(ByteString.copyFrom(entryBody.nioBuffer()));
readResponse.setBody(ByteBufUtil.getBytes(entryBody));
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Pass ByteBuf directly? Also ensure whether .retain() also needs to be called.

lastEntry = requestProcessor.bookie.readEntry(ledgerId, BookieProtocol.LAST_ADD_CONFIRMED);
if (lastEntry != null) {
readLacResponse.setLastEntryBody(ByteString.copyFrom(lastEntry.nioBuffer()));
readLacResponse.setLastEntryBody(ByteBufUtil.getBytes(lastEntry));
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Pass ByteBuf directly? Also ensure whether .retain() also needs to be called.

Copy link
Copy Markdown
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

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

LGTM

great!

Copy link
Copy Markdown
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

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

(I clicked on Approve by mistake)

I am happy with this improvement, but we have to solve the conflicts and fix CI

@eolivelli eolivelli self-requested a review May 8, 2026 10:33
merlimat added 3 commits May 8, 2026 10:08
…n-bkproto

# Conflicts:
#	bookkeeper-proto/pom.xml
#	bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java
#	buildtools/src/main/resources/bookkeeper/findbugsExclude.xml
Replace ByteBufList.coalesce(...) with a new ByteBufList#toByteBuf method
on the WriteLac and AddEntry request paths. coalesce allocates a new
buffer and copies all the bytes; toByteBuf wraps the existing buffers in
a CompositeByteBuf (or returns the single buffer directly when the list
has one entry, or Unpooled.EMPTY_BUFFER when empty), transferring
ownership to the caller and releasing the source ByteBufList.

Adds unit tests covering the empty / single / multi-buffer paths,
including ref-count behaviour for both the list and the underlying
buffers.
The previous implementation released the source ByteBufList when
producing the wrapping ByteBuf, but callers in PerChannelBookieClient
only wrap a buffer they don't own &mdash; the ByteBufList's lifecycle is
managed by the upstream PendingAddOp, which shares the same list across
multiple bookies in a quorum write. Releasing in toByteBuf produced a
double-release / use-after-free that surfaced as
IllegalReferenceCountException in tests like BookieStickyReadsTest.

Change toByteBuf to leave the source list's ref count untouched and
return a wrapper that holds its own retains (a CompositeByteBuf for
multiple buffers, a retainedDuplicate for the single-buffer fast path,
or Unpooled.EMPTY_BUFFER for empty). This matches the original
ByteBufList.coalesce semantics from the caller's point of view, while
still avoiding the byte copy.

Tests updated to assert the new ownership semantics.
Copy link
Copy Markdown
Member

@lhotari lhotari left a comment

Choose a reason for hiding this comment

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

LGTM

merlimat added 2 commits May 8, 2026 12:33
Four completion handlers (AddCompletion, ForceLedgerCompletion,
ReadCompletion, WriteLacCompletion) were always reading the inner
*Response's ledgerId/entryId fields, even when the outer Response
carried an error status. On error responses (e.g. EUA from a rejected
SASL handshake) those required fields are not populated. Under
protobuf-java they returned the default 0; under LightProto they throw
IllegalStateException("Field 'ledgerId' is not set"), which surfaced as
GSSAPIBookKeeperTest.testNotAllowedClientId blowing up on the client
side after the server rejected the auth.

Read the inner response only when status is EOK and hasXxxResponse() is
true. Otherwise fall back to the request's ledgerId/entryId, which the
CompletionValue base class already records.
Long-poll reads send entryId=LAST_ADD_CONFIRMED and the bookie fills in
the actual entry id (and ledgerId) on the response when an entry is
returned. The previous fix in ReadCompletion always used the request's
recorded entryId, which made the long-poll path look like an empty
piggy-back response on the client side and dropped the entry buffer.

Read ledgerId/entryId from the response when status is EOK and the
inner ReadResponse is present; fall back to the request's recorded
values only on error envelopes (where the inner response may be
missing or unpopulated). Fixes
TestReadLastConfirmedAndEntry.testRaceOnLastAddConfirmed.
@merlimat merlimat merged commit 47efc0d into master May 8, 2026
21 checks passed
@merlimat merlimat deleted the lightproto-migration-bkproto branch May 8, 2026 23:02
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.

3 participants