From 217e62b2b2df79dc81be35d24722a54f481ce1a5 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 7 May 2026 16:27:08 -0700 Subject: [PATCH 01/11] Fix LedgerHandle.batchReadUnconfirmedAsync: use slog log instead of LOG 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. --- .../main/java/org/apache/bookkeeper/client/LedgerHandle.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java index e9e63282ffd..0a1005f6ce3 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java @@ -982,7 +982,10 @@ public CompletableFuture batchReadAsync(long startEntry, int maxC public CompletableFuture batchReadUnconfirmedAsync(long startEntry, int maxCount, long maxSize) { // Little sanity check if (startEntry < 0 || maxCount < 0 || maxSize < 0) { - LOG.error("IncorrectParameterException on ledgerId:{} firstEntry:{} when batch read", ledgerId, startEntry); + log.error() + .attr("ledgerId", ledgerId) + .attr("startEntry", startEntry) + .log("IncorrectParameterException when batch read"); return FutureUtils.exception(new BKIncorrectParameterException()); } From 0c8c09adf8197801b865c92c2a6b2596996f67ce Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 7 May 2026 13:35:57 -0700 Subject: [PATCH 02/11] Migrate DataFormats and DbLedgerStorageDataFormats from protobuf-java 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`. --- bookkeeper-proto/pom.xml | 32 ++- .../org/apache/bookkeeper/bookie/Cookie.java | 29 +- .../storage/ldb/LedgerMetadataIndex.java | 31 +-- .../storage/ldb/LedgersIndexCheckOp.java | 10 +- .../storage/ldb/LedgersIndexRebuildOp.java | 10 +- .../ldb/SingleDirectoryDbLedgerStorage.java | 19 +- .../apache/bookkeeper/client/BookKeeper.java | 12 +- .../discover/ZKRegistrationClient.java | 36 +-- .../discover/ZKRegistrationManager.java | 41 +-- .../bookkeeper/meta/LedgerMetadataSerDe.java | 260 +++++++++--------- .../meta/ZkLedgerAuditorManager.java | 16 +- .../meta/ZkLedgerUnderreplicationManager.java | 108 ++++---- .../proto/checksum/DigestManager.java | 2 +- .../bookie/BookieInitializationTest.java | 7 +- .../apache/bookkeeper/client/ClientUtil.java | 2 +- .../client/ReadLastConfirmedOpTest.java | 2 +- .../apache/bookkeeper/proto/MockBookies.java | 3 +- .../TestLedgerUnderreplicationManager.java | 17 +- .../bookkeeper/test/BookieClientTest.java | 14 +- .../checksum/DigestManagerBenchmark.java | 2 +- .../proto/checksum/DigestTypeBenchmark.java | 2 +- pom.xml | 1 + 22 files changed, 331 insertions(+), 325 deletions(-) diff --git a/bookkeeper-proto/pom.xml b/bookkeeper-proto/pom.xml index 21e28cafce7..d997fbde438 100644 --- a/bookkeeper-proto/pom.xml +++ b/bookkeeper-proto/pom.xml @@ -29,6 +29,10 @@ com.google.protobuf protobuf-java + + io.netty + netty-buffer + @@ -37,10 +41,9 @@ apache-rat-plugin - - **/DataFormats.java + **/BookkeeperProtocol.java - **/DbLedgerStorageDataFormats.java + target/generated-sources/lightproto/** **/.checkstyle @@ -52,6 +55,9 @@ com.google.protobuf:protoc:${protoc.version}:exe:${os.detected.classifier} true + + BookkeeperProtocol.proto + @@ -61,6 +67,26 @@ + + io.streamnative.lightproto + lightproto-maven-plugin + ${lightproto-maven-plugin.version} + + + ${project.basedir}/src/main/proto/DataFormats.proto + ${project.basedir}/src/main/proto/DbLedgerStorageDataFormats.proto + + generated-sources/lightproto/java + true + + + + + generate + + + + diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java index b6a8795a6a4..d85e672ceaa 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java @@ -24,7 +24,6 @@ import com.google.common.base.Joiner; import com.google.common.collect.Sets; -import com.google.protobuf.TextFormat; import java.io.BufferedReader; import java.io.BufferedWriter; import java.io.EOFException; @@ -36,6 +35,7 @@ import java.io.InputStreamReader; import java.io.OutputStreamWriter; import java.io.StringReader; +import java.io.StringWriter; import java.net.InetSocketAddress; import java.net.UnknownHostException; import java.util.Objects; @@ -46,7 +46,7 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.discover.RegistrationManager; import org.apache.bookkeeper.net.BookieId; -import org.apache.bookkeeper.proto.DataFormats.CookieFormat; +import org.apache.bookkeeper.proto.CookieFormat; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.versioning.LongVersion; import org.apache.bookkeeper.versioning.Version; @@ -181,20 +181,20 @@ public String toString() { if (layoutVersion <= 3) { return toStringVersion3(); } - CookieFormat.Builder builder = CookieFormat.newBuilder(); - builder.setBookieHost(bookieId); - builder.setJournalDir(journalDirs); - builder.setLedgerDirs(ledgerDirs); + CookieFormat fmt = new CookieFormat(); + fmt.setBookieHost(bookieId); + fmt.setJournalDir(journalDirs); + fmt.setLedgerDirs(ledgerDirs); if (null != instanceId) { - builder.setInstanceId(instanceId); + fmt.setInstanceId(instanceId); } if (null != indexDirs) { - builder.setIndexDirs(indexDirs); + fmt.setIndexDirs(indexDirs); } StringBuilder b = new StringBuilder(); b.append(CURRENT_COOKIE_LAYOUT_VERSION).append("\n"); - b.append(builder.build()); + fmt.writeTextFormatTo(b); return b.toString(); } @@ -226,17 +226,18 @@ private static Builder parse(BufferedReader reader) throws IOException { cBuilder.setJournalDirs(reader.readLine()); cBuilder.setLedgerDirs(reader.readLine()); } else if (layoutVersion >= 4) { - CookieFormat.Builder cfBuilder = CookieFormat.newBuilder(); - TextFormat.merge(reader, cfBuilder); - CookieFormat data = cfBuilder.build(); + CookieFormat data = new CookieFormat(); + StringWriter rest = new StringWriter(); + reader.transferTo(rest); + data.parseFromTextFormat(rest.toString()); cBuilder.setBookieId(data.getBookieHost()); cBuilder.setJournalDirs(data.getJournalDir()); cBuilder.setLedgerDirs(data.getLedgerDirs()); // Since InstanceId is optional - if (null != data.getInstanceId() && !data.getInstanceId().isEmpty()) { + if (data.hasInstanceId() && !data.getInstanceId().isEmpty()) { cBuilder.setInstanceId(data.getInstanceId()); } - if (null != data.getIndexDirs() && !data.getIndexDirs().isEmpty()) { + if (data.hasIndexDirs() && !data.getIndexDirs().isEmpty()) { cBuilder.setIndexDirs(data.getIndexDirs()); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java index 7162c0c6c23..9037d4d0753 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java @@ -23,8 +23,8 @@ import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; -import com.google.protobuf.ByteString; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufUtil; import java.io.Closeable; import java.io.IOException; import java.util.AbstractMap.SimpleEntry; @@ -37,7 +37,6 @@ import lombok.CustomLog; import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieException; -import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorageDataFormats.LedgerData; import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorage.CloseableIterator; import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageFactory.DbConfigType; import org.apache.bookkeeper.conf.ServerConfiguration; @@ -82,7 +81,8 @@ public LedgerMetadataIndex(ServerConfiguration conf, KeyValueStorageFactory stor Entry entry = iterator.next(); long ledgerId = ArrayUtil.getLong(entry.getKey(), 0); if (ledgerId >= 0) { - LedgerData ledgerData = LedgerData.parseFrom(entry.getValue()); + LedgerData ledgerData = new LedgerData(); + ledgerData.parseFrom(entry.getValue()); ledgers.put(ledgerId, ledgerData); ledgersCount.incrementAndGet(); } @@ -119,7 +119,7 @@ public LedgerData get(long ledgerId) throws IOException { } public void set(long ledgerId, LedgerData ledgerData) throws IOException { - ledgerData = LedgerData.newBuilder(ledgerData).setExists(true).build(); + ledgerData = new LedgerData().copyFrom(ledgerData).setExists(true); ReentrantLock lock = lockForLedger(ledgerId); lock.lock(); @@ -170,11 +170,11 @@ public boolean setFenced(long ledgerId) throws IOException { lock.lock(); try { LedgerData ledgerData = get(ledgerId); - if (ledgerData.getFenced()) { + if (ledgerData.isFenced()) { return false; } - LedgerData newLedgerData = LedgerData.newBuilder(ledgerData).setFenced(true).build(); + LedgerData newLedgerData = new LedgerData().copyFrom(ledgerData).setFenced(true); if (ledgers.put(ledgerId, newLedgerData) == null) { // Ledger had been deleted @@ -197,11 +197,11 @@ public boolean setLimbo(long ledgerId) throws IOException { lock.lock(); try { LedgerData ledgerData = get(ledgerId); - if (ledgerData.getLimbo()) { + if (ledgerData.isLimbo()) { return false; } - LedgerData newLedgerData = LedgerData.newBuilder(ledgerData).setLimbo(true).build(); + LedgerData newLedgerData = new LedgerData().copyFrom(ledgerData).setLimbo(true); if (ledgers.put(ledgerId, newLedgerData) == null) { // Ledger had been deleted @@ -227,8 +227,8 @@ public boolean clearLimbo(long ledgerId) throws IOException { if (ledgerData == null) { throw new Bookie.NoLedgerException(ledgerId); } - final boolean oldValue = ledgerData.getLimbo(); - LedgerData newLedgerData = LedgerData.newBuilder(ledgerData).setLimbo(false).build(); + final boolean oldValue = ledgerData.isLimbo(); + LedgerData newLedgerData = new LedgerData().copyFrom(ledgerData).setLimbo(false); if (ledgers.put(ledgerId, newLedgerData) == null) { // Ledger had been deleted @@ -254,14 +254,13 @@ public void setMasterKey(long ledgerId, byte[] masterKey) throws IOException { LedgerData ledgerData = ledgers.get(ledgerId); if (ledgerData == null) { // New ledger inserted - ledgerData = LedgerData.newBuilder().setExists(true).setFenced(false) - .setMasterKey(ByteString.copyFrom(masterKey)).build(); + ledgerData = new LedgerData().setExists(true).setFenced(false).setMasterKey(masterKey); log.debug().attr("ledgerId", ledgerId).log("Inserting new ledger"); } else { - byte[] storedMasterKey = ledgerData.getMasterKey().toByteArray(); + byte[] storedMasterKey = ledgerData.getMasterKey(); if (ArrayUtil.isArrayAllZeros(storedMasterKey)) { // update master key of the ledger - ledgerData = LedgerData.newBuilder(ledgerData).setMasterKey(ByteString.copyFrom(masterKey)).build(); + ledgerData = new LedgerData().copyFrom(ledgerData).setMasterKey(masterKey); log.debug() .attr("storedMasterKey", storedMasterKey) .attr("newMasterKey", masterKey) @@ -388,8 +387,8 @@ boolean setStorageStateFlags(int expected, int newFlags) throws IOException { void setExplicitLac(long ledgerId, ByteBuf lac) throws IOException { LedgerData ledgerData = ledgers.get(ledgerId); if (ledgerData != null) { - LedgerData newLedgerData = LedgerData.newBuilder(ledgerData) - .setExplicitLac(ByteString.copyFrom(lac.nioBuffer())).build(); + LedgerData newLedgerData = new LedgerData().copyFrom(ledgerData) + .setExplicitLac(ByteBufUtil.getBytes(lac)); if (ledgers.put(ledgerId, newLedgerData) == null) { // Ledger had been deleted diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexCheckOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexCheckOp.java index 8a7df8f5652..c9cae7ddff7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexCheckOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexCheckOp.java @@ -77,17 +77,17 @@ public boolean initiate() throws IOException { ctr++; Map.Entry entry = iterator.next(); long ledgerId = ArrayUtil.getLong(entry.getKey(), 0); - DbLedgerStorageDataFormats.LedgerData ledgerData = - DbLedgerStorageDataFormats.LedgerData.parseFrom(entry.getValue()); + LedgerData ledgerData = new LedgerData(); + ledgerData.parseFrom(entry.getValue()); if (verbose) { log.info() .attr("scanned", ctr) .attr("ledgerId", ledgerId) - .attr("exists", (ledgerData.hasExists() ? ledgerData.getExists() : "-")) - .attr("isFenced", (ledgerData.hasFenced() ? ledgerData.getFenced() : "-")) + .attr("exists", (ledgerData.hasExists() ? ledgerData.isExists() : "-")) + .attr("isFenced", (ledgerData.hasFenced() ? ledgerData.isFenced() : "-")) .attr("masterKey", (ledgerData.hasMasterKey() ? Base64.getEncoder() - .encodeToString(ledgerData.getMasterKey().toByteArray()) + .encodeToString(ledgerData.getMasterKey()) : "-")) .attr("explicitLAC", (ledgerData.hasExplicitLac() ? ledgerData.getExplicitLac() : "-")) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildOp.java index fc9f94559a8..960d9cd8c17 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildOp.java @@ -21,7 +21,6 @@ package org.apache.bookkeeper.bookie.storage.ldb; import com.google.common.collect.Lists; -import com.google.protobuf.ByteString; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; @@ -109,11 +108,10 @@ public boolean initiate() { log.info().attr("tempPath", indexTempPath).log("Created ledgers index at temp location"); for (Long ledgerId : ledgers) { - DbLedgerStorageDataFormats.LedgerData ledgerData = - DbLedgerStorageDataFormats.LedgerData.newBuilder() - .setExists(true) - .setFenced(true) - .setMasterKey(ByteString.EMPTY).build(); + LedgerData ledgerData = new LedgerData() + .setExists(true) + .setFenced(true) + .setMasterKey(new byte[0]); byte[] ledgerArray = new byte[16]; ArrayUtil.setLong(ledgerArray, 0, ledgerId); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java index 708b63da62b..6778de2f824 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java @@ -26,7 +26,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.google.protobuf.ByteString; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; @@ -68,7 +67,6 @@ import org.apache.bookkeeper.bookie.LedgerEntryPage; import org.apache.bookkeeper.bookie.StateManager; import org.apache.bookkeeper.bookie.storage.EntryLogger; -import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorageDataFormats.LedgerData; import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorage.Batch; import org.apache.bookkeeper.common.util.MathUtils; import org.apache.bookkeeper.common.util.Watcher; @@ -372,9 +370,9 @@ public boolean ledgerExists(long ledgerId) throws IOException { LedgerData ledgerData = ledgerIndex.get(ledgerId); log.debug() .attr("ledgerId", ledgerId) - .attr("exists", () -> ledgerData.getExists()) + .attr("exists", () -> ledgerData.isExists()) .log("Ledger exists"); - return ledgerData.getExists(); + return ledgerData.isExists(); } catch (Bookie.NoLedgerException nle) { // ledger does not exist return false; @@ -426,7 +424,7 @@ public boolean entryExists(long ledgerId, long entryId) throws IOException, Book @Override public boolean isFenced(long ledgerId) throws IOException, BookieException { - boolean isFenced = ledgerIndex.get(ledgerId).getFenced(); + boolean isFenced = ledgerIndex.get(ledgerId).isFenced(); log.debug() .attr("ledgerId", ledgerId) @@ -464,7 +462,7 @@ public void setMasterKey(long ledgerId, byte[] masterKey) throws IOException { @Override public byte[] readMasterKey(long ledgerId) throws IOException, BookieException { log.debug().attr("ledgerId", ledgerId).log("Read master key"); - return ledgerIndex.get(ledgerId).getMasterKey().toByteArray(); + return ledgerIndex.get(ledgerId).getMasterKey(); } @Override @@ -1037,8 +1035,7 @@ public ByteBuf getExplicitLac(long ledgerId) throws IOException, BookieException return null; } log.debug().attr("ledgerId", ledgerId).log("getExplicitLac returned from LedgerData"); - ByteString persistedLac = ledgerData.getExplicitLac(); - ledgerInfo.setExplicitLac(Unpooled.wrappedBuffer(persistedLac.toByteArray())); + ledgerInfo.setExplicitLac(Unpooled.wrappedBuffer(ledgerData.getExplicitLac())); return ledgerInfo.getExplicitLac(); } @@ -1074,8 +1071,8 @@ public void flushEntriesLocationsIndex() throws IOException { @SuppressFBWarnings("RCN_REDUNDANT_NULLCHECK_WOULD_HAVE_BEEN_A_NPE") public long addLedgerToIndex(long ledgerId, boolean isFenced, byte[] masterKey, LedgerCache.PageEntriesIterable pages) throws Exception { - LedgerData ledgerData = LedgerData.newBuilder().setExists(true).setFenced(isFenced) - .setMasterKey(ByteString.copyFrom(masterKey)).build(); + LedgerData ledgerData = new LedgerData().setExists(true).setFenced(isFenced) + .setMasterKey(masterKey); ledgerIndex.set(ledgerId, ledgerData); MutableLong numberOfEntries = new MutableLong(); @@ -1217,7 +1214,7 @@ public void setLimboState(long ledgerId) throws IOException { @Override public boolean hasLimboState(long ledgerId) throws IOException { log.debug().attr("ledgerId", ledgerId).log("hasLimboState"); - return ledgerIndex.get(ledgerId).getLimbo(); + return ledgerIndex.get(ledgerId).isLimbo(); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java index e34a2ebefa4..db34b482f24 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java @@ -85,7 +85,7 @@ import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookieClientImpl; -import org.apache.bookkeeper.proto.DataFormats; +import org.apache.bookkeeper.proto.LedgerMetadataFormat; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.EventLoopUtil; @@ -738,16 +738,16 @@ public static DigestType fromApiDigestType(org.apache.bookkeeper.client.api.Dige throw new IllegalArgumentException("Unable to convert digest type " + digestType); } } - public static DataFormats.LedgerMetadataFormat.DigestType toProtoDigestType(DigestType digestType) { + public static LedgerMetadataFormat.DigestType toProtoDigestType(DigestType digestType) { switch (digestType) { case MAC: - return DataFormats.LedgerMetadataFormat.DigestType.HMAC; + return LedgerMetadataFormat.DigestType.HMAC; case CRC32: - return DataFormats.LedgerMetadataFormat.DigestType.CRC32; + return LedgerMetadataFormat.DigestType.CRC32; case CRC32C: - return DataFormats.LedgerMetadataFormat.DigestType.CRC32C; + return LedgerMetadataFormat.DigestType.CRC32C; case DUMMY: - return DataFormats.LedgerMetadataFormat.DigestType.DUMMY; + return LedgerMetadataFormat.DigestType.DUMMY; default: throw new IllegalArgumentException("Unable to convert digest type " + digestType); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java index f0ff9121c32..8622f667bdb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java @@ -26,8 +26,10 @@ import com.google.common.collect.Sets; import java.io.IOException; import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -36,7 +38,6 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; -import java.util.stream.Collectors; import lombok.AccessLevel; import lombok.CustomLog; import lombok.Getter; @@ -44,7 +45,7 @@ import org.apache.bookkeeper.client.BKException.ZKException; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.net.BookieId; -import org.apache.bookkeeper.proto.DataFormats.BookieServiceInfoFormat; +import org.apache.bookkeeper.proto.BookieServiceInfoFormat; import org.apache.bookkeeper.versioning.LongVersion; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Version.Occurred; @@ -322,23 +323,26 @@ static BookieServiceInfo deserializeBookieServiceInfo(BookieId bookieId, byte[] return BookieServiceInfoUtils.buildLegacyBookieServiceInfo(bookieId.toString()); } - BookieServiceInfoFormat builder = BookieServiceInfoFormat.parseFrom(bookieServiceInfo); + BookieServiceInfoFormat fmt = new BookieServiceInfoFormat(); + fmt.parseFrom(bookieServiceInfo); BookieServiceInfo bsi = new BookieServiceInfo(); - List endpoints = builder.getEndpointsList().stream() - .map(e -> { - BookieServiceInfo.Endpoint endpoint = new BookieServiceInfo.Endpoint(); - endpoint.setId(e.getId()); - endpoint.setPort(e.getPort()); - endpoint.setHost(e.getHost()); - endpoint.setProtocol(e.getProtocol()); - endpoint.setAuth(e.getAuthList()); - endpoint.setExtensions(e.getExtensionsList()); - return endpoint; - }) - .collect(Collectors.toList()); + List endpoints = new ArrayList<>(fmt.getEndpointsCount()); + for (int i = 0; i < fmt.getEndpointsCount(); i++) { + BookieServiceInfoFormat.Endpoint e = fmt.getEndpointAt(i); + BookieServiceInfo.Endpoint endpoint = new BookieServiceInfo.Endpoint(); + endpoint.setId(e.getId()); + endpoint.setPort(e.getPort()); + endpoint.setHost(e.getHost()); + endpoint.setProtocol(e.getProtocol()); + endpoint.setAuth(e.getAuthsList()); + endpoint.setExtensions(e.getExtensionsList()); + endpoints.add(endpoint); + } bsi.setEndpoints(endpoints); - bsi.setProperties(builder.getPropertiesMap()); + Map properties = new HashMap<>(); + fmt.forEachProperties(properties::put); + bsi.setProperties(properties); return bsi; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java index 80a45c0dd7b..84827125cab 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationManager.java @@ -27,7 +27,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; -import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -37,7 +36,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.function.Function; -import java.util.stream.Collectors; import lombok.CustomLog; import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.BookieException.BookieIllegalOpException; @@ -56,7 +54,7 @@ import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieId; -import org.apache.bookkeeper.proto.DataFormats.BookieServiceInfoFormat; +import org.apache.bookkeeper.proto.BookieServiceInfoFormat; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.util.ZkUtils; import org.apache.bookkeeper.versioning.LongVersion; @@ -239,33 +237,18 @@ public void registerBookie(BookieId bookieId, boolean readOnly, @VisibleForTesting static byte[] serializeBookieServiceInfo(BookieServiceInfo bookieServiceInfo) { log.debug().attr("bookieServiceInfo", bookieServiceInfo).log("serialize BookieServiceInfo"); - try (ByteArrayOutputStream os = new ByteArrayOutputStream()) { - BookieServiceInfoFormat.Builder builder = BookieServiceInfoFormat.newBuilder(); - List bsiEndpoints = bookieServiceInfo.getEndpoints().stream() - .map(e -> { - return BookieServiceInfoFormat.Endpoint.newBuilder() - .setId(e.getId()) - .setPort(e.getPort()) - .setHost(e.getHost()) - .setProtocol(e.getProtocol()) - .addAllAuth(e.getAuth()) - .addAllExtensions(e.getExtensions()) - .build(); - }) - .collect(Collectors.toList()); - - builder.addAllEndpoints(bsiEndpoints); - builder.putAllProperties(bookieServiceInfo.getProperties()); - - builder.build().writeTo(os); - return os.toByteArray(); - } catch (IOException err) { - log.error() - .exception(err) - .attr("bookieServiceInfo", bookieServiceInfo) - .log("Cannot serialize bookieServiceInfo"); - throw new RuntimeException(err); + BookieServiceInfoFormat fmt = new BookieServiceInfoFormat(); + for (BookieServiceInfo.Endpoint e : bookieServiceInfo.getEndpoints()) { + BookieServiceInfoFormat.Endpoint endpoint = fmt.addEndpoint() + .setId(e.getId()) + .setPort(e.getPort()) + .setHost(e.getHost()) + .setProtocol(e.getProtocol()); + endpoint.addAllAuths(e.getAuth()); + endpoint.addAllExtensions(e.getExtensions()); } + bookieServiceInfo.getProperties().forEach(fmt::putProperties); + return fmt.toByteArray(); } private void doRegisterBookie(String regPath, BookieServiceInfo bookieServiceInfo) throws BookieException { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java index bcbbf8473a1..e8c24ba5803 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/LedgerMetadataSerDe.java @@ -21,11 +21,12 @@ import static com.google.common.base.Preconditions.checkState; import static java.nio.charset.StandardCharsets.UTF_8; -import com.google.protobuf.ByteString; -import com.google.protobuf.TextFormat; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; import java.io.BufferedReader; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; +import java.io.EOFException; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; @@ -35,10 +36,10 @@ import java.io.UnsupportedEncodingException; import java.util.ArrayList; import java.util.Base64; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.stream.Collectors; import lombok.CustomLog; import org.apache.bookkeeper.client.LedgerMetadataBuilder; import org.apache.bookkeeper.client.LedgerMetadataUtils; @@ -46,7 +47,7 @@ import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.LedgerMetadata.State; import org.apache.bookkeeper.net.BookieId; -import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat; +import org.apache.bookkeeper.proto.LedgerMetadataFormat; /** * Serialization and deserialization for LedgerMetadata. @@ -152,61 +153,8 @@ public byte[] serialize(LedgerMetadata metadata) throws IOException { private static byte[] serializeVersion3(LedgerMetadata metadata) throws IOException { try (ByteArrayOutputStream os = new ByteArrayOutputStream()) { writeHeader(os, METADATA_FORMAT_VERSION_3); - LedgerMetadataFormat.Builder builder = LedgerMetadataFormat.newBuilder(); - builder.setQuorumSize(metadata.getWriteQuorumSize()) - .setAckQuorumSize(metadata.getAckQuorumSize()) - .setEnsembleSize(metadata.getEnsembleSize()) - .setLength(metadata.getLength()) - .setLastEntryId(metadata.getLastEntryId()); - - switch (metadata.getState()) { - case CLOSED: - builder.setState(LedgerMetadataFormat.State.CLOSED); - break; - case IN_RECOVERY: - builder.setState(LedgerMetadataFormat.State.IN_RECOVERY); - break; - case OPEN: - builder.setState(LedgerMetadataFormat.State.OPEN); - break; - default: - checkArgument(false, - String.format("Unknown state %s for protobuf serialization", metadata.getState())); - break; - } - - /** Hack to get around fact that ctime was never versioned correctly */ - if (LedgerMetadataUtils.shouldStoreCtime(metadata)) { - builder.setCtime(metadata.getCtime()); - } - - - builder.setDigestType(apiToProtoDigestType(metadata.getDigestType())); - - serializePassword(metadata.getPassword(), builder); - - Map customMetadata = metadata.getCustomMetadata(); - if (customMetadata.size() > 0) { - LedgerMetadataFormat.cMetadataMapEntry.Builder cMetadataBuilder = - LedgerMetadataFormat.cMetadataMapEntry.newBuilder(); - for (Map.Entry entry : customMetadata.entrySet()) { - cMetadataBuilder.setKey(entry.getKey()).setValue(ByteString.copyFrom(entry.getValue())); - builder.addCustomMetadata(cMetadataBuilder.build()); - } - } - - for (Map.Entry> entry : metadata.getAllEnsembles().entrySet()) { - LedgerMetadataFormat.Segment.Builder segmentBuilder = LedgerMetadataFormat.Segment.newBuilder(); - segmentBuilder.setFirstEntryId(entry.getKey()); - for (BookieId addr : entry.getValue()) { - segmentBuilder.addEnsembleMember(addr.toString()); - } - builder.addSegment(segmentBuilder.build()); - } - - builder.setCToken(metadata.getCToken()); - - builder.build().writeDelimitedTo(os); + LedgerMetadataFormat fmt = buildLedgerMetadataFormat(metadata, true); + writeDelimited(fmt, os); return os.toByteArray(); } } @@ -221,62 +169,62 @@ private static byte[] serializeVersion2(LedgerMetadata metadata) throws IOExcept * fields, and if this code was shared with version 3, it would be easy to * accidently add new fields and create BC issues. **********************************************************************/ - LedgerMetadataFormat.Builder builder = LedgerMetadataFormat.newBuilder(); - builder.setQuorumSize(metadata.getWriteQuorumSize()) - .setAckQuorumSize(metadata.getAckQuorumSize()) - .setEnsembleSize(metadata.getEnsembleSize()) - .setLength(metadata.getLength()) - .setLastEntryId(metadata.getLastEntryId()); - - switch (metadata.getState()) { - case CLOSED: - builder.setState(LedgerMetadataFormat.State.CLOSED); - break; - case IN_RECOVERY: - builder.setState(LedgerMetadataFormat.State.IN_RECOVERY); - break; - case OPEN: - builder.setState(LedgerMetadataFormat.State.OPEN); - break; - default: - checkArgument(false, - String.format("Unknown state %s for protobuf serialization", metadata.getState())); - break; - } + LedgerMetadataFormat fmt = buildLedgerMetadataFormat(metadata, false); + writer.print(fmt.toTextFormat()); + writer.flush(); + } + return os.toByteArray(); + } + } - /** Hack to get around fact that ctime was never versioned correctly */ - if (LedgerMetadataUtils.shouldStoreCtime(metadata)) { - builder.setCtime(metadata.getCtime()); - } + private static LedgerMetadataFormat buildLedgerMetadataFormat(LedgerMetadata metadata, boolean includeCToken) { + LedgerMetadataFormat fmt = new LedgerMetadataFormat() + .setQuorumSize(metadata.getWriteQuorumSize()) + .setAckQuorumSize(metadata.getAckQuorumSize()) + .setEnsembleSize(metadata.getEnsembleSize()) + .setLength(metadata.getLength()) + .setLastEntryId(metadata.getLastEntryId()); - builder.setDigestType(apiToProtoDigestType(metadata.getDigestType())); - serializePassword(metadata.getPassword(), builder); + switch (metadata.getState()) { + case CLOSED: + fmt.setState(LedgerMetadataFormat.State.CLOSED); + break; + case IN_RECOVERY: + fmt.setState(LedgerMetadataFormat.State.IN_RECOVERY); + break; + case OPEN: + fmt.setState(LedgerMetadataFormat.State.OPEN); + break; + default: + checkArgument(false, + String.format("Unknown state %s for protobuf serialization", metadata.getState())); + break; + } - Map customMetadata = metadata.getCustomMetadata(); - if (customMetadata.size() > 0) { - LedgerMetadataFormat.cMetadataMapEntry.Builder cMetadataBuilder = - LedgerMetadataFormat.cMetadataMapEntry.newBuilder(); - for (Map.Entry entry : customMetadata.entrySet()) { - cMetadataBuilder.setKey(entry.getKey()).setValue(ByteString.copyFrom(entry.getValue())); - builder.addCustomMetadata(cMetadataBuilder.build()); - } - } + /** Hack to get around fact that ctime was never versioned correctly */ + if (LedgerMetadataUtils.shouldStoreCtime(metadata)) { + fmt.setCtime(metadata.getCtime()); + } - for (Map.Entry> entry : - metadata.getAllEnsembles().entrySet()) { - LedgerMetadataFormat.Segment.Builder segmentBuilder = LedgerMetadataFormat.Segment.newBuilder(); - segmentBuilder.setFirstEntryId(entry.getKey()); - for (BookieId addr : entry.getValue()) { - segmentBuilder.addEnsembleMember(addr.toString()); - } - builder.addSegment(segmentBuilder.build()); - } + fmt.setDigestType(apiToProtoDigestType(metadata.getDigestType())); + serializePassword(metadata.getPassword(), fmt); - TextFormat.printer().print(builder.build(), writer); - writer.flush(); + for (Map.Entry entry : metadata.getCustomMetadata().entrySet()) { + fmt.addCustomMetadata().setKey(entry.getKey()).setValue(entry.getValue()); + } + + for (Map.Entry> entry : metadata.getAllEnsembles().entrySet()) { + LedgerMetadataFormat.Segment seg = fmt.addSegment(); + seg.setFirstEntryId(entry.getKey()); + for (BookieId addr : entry.getValue()) { + seg.addEnsembleMember(addr.toString()); } - return os.toByteArray(); } + + if (includeCToken) { + fmt.setCToken(metadata.getCToken()); + } + return fmt; } private static byte[] serializeVersion1(LedgerMetadata metadata) throws IOException { @@ -314,11 +262,11 @@ private static byte[] serializeVersion1(LedgerMetadata metadata) throws IOExcept } } - private static void serializePassword(byte[] password, LedgerMetadataFormat.Builder builder) { + private static void serializePassword(byte[] password, LedgerMetadataFormat fmt) { if (password == null || password.length == 0) { - builder.setPassword(ByteString.EMPTY); + fmt.setPassword(new byte[0]); } else { - builder.setPassword(ByteString.copyFrom(password)); + fmt.setPassword(password); } } @@ -368,9 +316,8 @@ private static LedgerMetadata parseVersion3Config(long ledgerId, InputStream is, LedgerMetadataBuilder builder = LedgerMetadataBuilder.create() .withId(ledgerId) .withMetadataFormatVersion(METADATA_FORMAT_VERSION_3); - LedgerMetadataFormat.Builder formatBuilder = LedgerMetadataFormat.newBuilder(); - formatBuilder.mergeDelimitedFrom(is); - LedgerMetadataFormat data = formatBuilder.build(); + LedgerMetadataFormat data = new LedgerMetadataFormat(); + readDelimited(is, data); decodeFormat(data, builder); if (data.hasCtime()) { builder.storingCreationTime(true); @@ -386,11 +333,8 @@ private static LedgerMetadata parseVersion2Config(long ledgerId, InputStream is, .withId(ledgerId) .withMetadataFormatVersion(METADATA_FORMAT_VERSION_2); - LedgerMetadataFormat.Builder formatBuilder = LedgerMetadataFormat.newBuilder(); - try (InputStreamReader reader = new InputStreamReader(is, UTF_8.name())) { - TextFormat.merge(reader, formatBuilder); - } - LedgerMetadataFormat data = formatBuilder.build(); + LedgerMetadataFormat data = new LedgerMetadataFormat(); + data.parseFromTextFormat(is.readAllBytes()); decodeFormat(data, builder); if (data.hasCtime()) { // 'storingCreationTime' is only ever taken into account for serializing version 2 @@ -421,22 +365,26 @@ private static void decodeFormat(LedgerMetadataFormat data, LedgerMetadataBuilde } if (data.hasPassword()) { - builder.withPassword(data.getPassword().toByteArray()) + builder.withPassword(data.getPassword()) .withDigestType(protoToApiDigestType(data.getDigestType())); } - for (LedgerMetadataFormat.Segment s : data.getSegmentList()) { - List addrs = new ArrayList<>(); - for (String addr : s.getEnsembleMemberList()) { - addrs.add(BookieId.parse(addr)); + for (int i = 0; i < data.getSegmentsCount(); i++) { + LedgerMetadataFormat.Segment s = data.getSegmentAt(i); + List addrs = new ArrayList<>(s.getEnsembleMembersCount()); + for (int j = 0; j < s.getEnsembleMembersCount(); j++) { + addrs.add(BookieId.parse(s.getEnsembleMemberAt(j))); } builder.newEnsembleEntry(s.getFirstEntryId(), addrs); } - if (data.getCustomMetadataCount() > 0) { - builder.withCustomMetadata(data.getCustomMetadataList().stream().collect( - Collectors.toMap(e -> e.getKey(), - e -> e.getValue().toByteArray()))); + if (data.getCustomMetadatasCount() > 0) { + Map customMetadata = new HashMap<>(); + for (int i = 0; i < data.getCustomMetadatasCount(); i++) { + LedgerMetadataFormat.cMetadataMapEntry e = data.getCustomMetadataAt(i); + customMetadata.put(e.getKey(), e.getValue()); + } + builder.withCustomMetadata(customMetadata); } if (data.hasCToken()) { @@ -512,4 +460,60 @@ private static DigestType protoToApiDigestType(LedgerMetadataFormat.DigestType d throw new IllegalArgumentException("Unable to convert digest type " + digestType); } } + + /** Write a length-prefixed message in protobuf-compatible delimited format. */ + private static void writeDelimited(LedgerMetadataFormat msg, OutputStream os) throws IOException { + int size = msg.getSerializedSize(); + writeRawVarint32(size, os); + ByteBuf buf = Unpooled.buffer(size); + try { + msg.writeTo(buf); + os.write(buf.array(), buf.arrayOffset() + buf.readerIndex(), buf.readableBytes()); + } finally { + buf.release(); + } + } + + /** Read a length-prefixed message in protobuf-compatible delimited format. */ + private static void readDelimited(InputStream is, LedgerMetadataFormat msg) throws IOException { + int size = readRawVarint32(is); + byte[] data = new byte[size]; + int read = 0; + while (read < size) { + int n = is.read(data, read, size - read); + if (n < 0) { + throw new EOFException("Unexpected EOF reading delimited message"); + } + read += n; + } + msg.parseFrom(data); + } + + private static void writeRawVarint32(int value, OutputStream os) throws IOException { + while (true) { + if ((value & ~0x7F) == 0) { + os.write(value); + return; + } + os.write((value & 0x7F) | 0x80); + value >>>= 7; + } + } + + private static int readRawVarint32(InputStream is) throws IOException { + int result = 0; + int shift = 0; + while (shift < 32) { + int b = is.read(); + if (b == -1) { + throw new EOFException("Unexpected EOF reading varint"); + } + result |= (b & 0x7F) << shift; + if ((b & 0x80) == 0) { + return result; + } + shift += 7; + } + throw new IOException("Malformed varint32"); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerAuditorManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerAuditorManager.java index 768b2663cbf..e174cc7b4f6 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerAuditorManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerAuditorManager.java @@ -18,10 +18,8 @@ package org.apache.bookkeeper.meta; import static java.nio.charset.StandardCharsets.UTF_8; -import static org.apache.bookkeeper.proto.DataFormats.AuditorVoteFormat; import static org.apache.bookkeeper.replication.ReplicationStats.ELECTION_ATTEMPTS; -import com.google.protobuf.TextFormat; import java.io.IOException; import java.io.Serializable; import java.util.Collections; @@ -33,6 +31,7 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.proto.AuditorVoteFormat; import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.stats.annotations.StatsDoc; @@ -112,11 +111,11 @@ public void tryToBecomeAuditor(String bookieId, Consumer listener) // We have been elected as the auditor // update the auditor bookie id in the election path. This is // done for debugging purpose - AuditorVoteFormat.Builder builder = AuditorVoteFormat.newBuilder() + AuditorVoteFormat fmt = new AuditorVoteFormat() .setBookieId(bookieId); zkc.setData(getVotePath(""), - builder.build().toString().getBytes(UTF_8), -1); + fmt.toTextFormat().getBytes(UTF_8), -1); return; } else { // If not an auditor, will be watching to my predecessor and @@ -162,9 +161,8 @@ public BookieId getCurrentAuditor() throws IOException, InterruptedException { String ledger = electionRoot + "/" + children.get(AUDITOR_INDEX); byte[] data = zkc.getData(ledger, false, null); - AuditorVoteFormat.Builder builder = AuditorVoteFormat.newBuilder(); - TextFormat.merge(new String(data, UTF_8), builder); - AuditorVoteFormat v = builder.build(); + AuditorVoteFormat v = new AuditorVoteFormat(); + v.parseFromTextFormat(data); return BookieId.parse(v.getBookieId()); } catch (KeeperException e) { throw new IOException(e); @@ -191,13 +189,13 @@ public void close() throws Exception { private void createMyVote(String bookieId) throws IOException, InterruptedException { List zkAcls = ZkUtils.getACLs(conf); - AuditorVoteFormat.Builder builder = AuditorVoteFormat.newBuilder() + AuditorVoteFormat fmt = new AuditorVoteFormat() .setBookieId(bookieId); try { if (null == myVote || null == zkc.exists(myVote, false)) { myVote = zkc.create(getVotePath(PATH_SEPARATOR + VOTE_PREFIX), - builder.build().toString().getBytes(UTF_8), zkAcls, + fmt.toTextFormat().getBytes(UTF_8), zkAcls, CreateMode.EPHEMERAL_SEQUENTIAL); } } catch (KeeperException e) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java index 7bc6ccf9b82..00e0d77d434 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java @@ -22,9 +22,6 @@ import com.google.common.base.Joiner; import com.google.common.util.concurrent.RateLimiter; -import com.google.protobuf.InvalidProtocolBufferException; -import com.google.protobuf.TextFormat; -import com.google.protobuf.TextFormat.ParseException; import java.net.UnknownHostException; import java.util.Arrays; import java.util.Collection; @@ -47,12 +44,12 @@ import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.DNS; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; -import org.apache.bookkeeper.proto.DataFormats.CheckAllLedgersFormat; -import org.apache.bookkeeper.proto.DataFormats.LedgerRereplicationLayoutFormat; -import org.apache.bookkeeper.proto.DataFormats.LockDataFormat; -import org.apache.bookkeeper.proto.DataFormats.PlacementPolicyCheckFormat; -import org.apache.bookkeeper.proto.DataFormats.ReplicasCheckFormat; -import org.apache.bookkeeper.proto.DataFormats.UnderreplicatedLedgerFormat; +import org.apache.bookkeeper.proto.CheckAllLedgersFormat; +import org.apache.bookkeeper.proto.LedgerRereplicationLayoutFormat; +import org.apache.bookkeeper.proto.LockDataFormat; +import org.apache.bookkeeper.proto.PlacementPolicyCheckFormat; +import org.apache.bookkeeper.proto.ReplicasCheckFormat; +import org.apache.bookkeeper.proto.UnderreplicatedLedgerFormat; import org.apache.bookkeeper.replication.ReplicationEnableCb; import org.apache.bookkeeper.replication.ReplicationException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; @@ -173,14 +170,14 @@ public static String getUrLockPath(String rootPath) { } public static byte[] getLockData() { - LockDataFormat.Builder lockDataBuilder = LockDataFormat.newBuilder(); + LockDataFormat lockData = new LockDataFormat(); try { - lockDataBuilder.setBookieId(DNS.getDefaultHost("default")); + lockData.setBookieId(DNS.getDefaultHost("default")); } catch (UnknownHostException uhe) { // if we cant get the address, ignore. it's optional // in the data structure in any case } - return lockDataBuilder.build().toString().getBytes(UTF_8); + return lockData.toTextFormat().getBytes(UTF_8); } private void checkLayout() @@ -195,10 +192,10 @@ private void checkLayout() } while (true) { if (zkc.exists(layoutZNode, false) == null) { - LedgerRereplicationLayoutFormat.Builder builder = LedgerRereplicationLayoutFormat.newBuilder(); - builder.setType(LAYOUT).setVersion(LAYOUT_VERSION); + LedgerRereplicationLayoutFormat layout = new LedgerRereplicationLayoutFormat(); + layout.setType(LAYOUT).setVersion(LAYOUT_VERSION); try { - zkc.create(layoutZNode, builder.build().toString().getBytes(UTF_8), + zkc.create(layoutZNode, layout.toTextFormat().getBytes(UTF_8), zkAcls, CreateMode.PERSISTENT); break; } catch (KeeperException.NodeExistsException nne) { @@ -207,17 +204,16 @@ private void checkLayout() } else { byte[] layoutData = zkc.getData(layoutZNode, false, null); - LedgerRereplicationLayoutFormat.Builder builder = LedgerRereplicationLayoutFormat.newBuilder(); + LedgerRereplicationLayoutFormat layout = new LedgerRereplicationLayoutFormat(); try { - TextFormat.merge(new String(layoutData, UTF_8), builder); - LedgerRereplicationLayoutFormat layout = builder.build(); + layout.parseFromTextFormat(layoutData); if (!layout.getType().equals(LAYOUT) || layout.getVersion() != LAYOUT_VERSION) { throw new ReplicationException.CompatibilityException( "Incompatible layout found (" + LAYOUT + ":" + LAYOUT_VERSION + ")"); } - } catch (TextFormat.ParseException pe) { + } catch (RuntimeException pe) { throw new ReplicationException.CompatibilityException( "Invalid data found", pe); } @@ -276,7 +272,7 @@ public UnderreplicatedLedger getLedgerUnreplicationInfo(long ledgerId) throws ReplicationException.UnavailableException { try { String znode = getUrLedgerZnode(ledgerId); - UnderreplicatedLedgerFormat.Builder builder = UnderreplicatedLedgerFormat.newBuilder(); + UnderreplicatedLedgerFormat fmt = new UnderreplicatedLedgerFormat(); byte[] data = null; try { data = zkc.getData(znode, false, null); @@ -284,11 +280,10 @@ public UnderreplicatedLedger getLedgerUnreplicationInfo(long ledgerId) log.debug().attr("ledgerId", ledgerId).log("Ledger is not marked underreplicated"); return null; } - TextFormat.merge(new String(data, UTF_8), builder); - UnderreplicatedLedgerFormat underreplicatedLedgerFormat = builder.build(); + fmt.parseFromTextFormat(data); UnderreplicatedLedger underreplicatedLedger = new UnderreplicatedLedger(ledgerId); - List replicaList = underreplicatedLedgerFormat.getReplicaList(); - long ctime = (underreplicatedLedgerFormat.hasCtime() ? underreplicatedLedgerFormat.getCtime() + List replicaList = fmt.getReplicasList(); + long ctime = (fmt.hasCtime() ? fmt.getCtime() : UnderreplicatedLedger.UNASSIGNED_CTIME); underreplicatedLedger.setCtime(ctime); underreplicatedLedger.setReplicaList(replicaList); @@ -298,7 +293,7 @@ public UnderreplicatedLedger getLedgerUnreplicationInfo(long ledgerId) } catch (InterruptedException ie) { Thread.currentThread().interrupt(); throw new ReplicationException.UnavailableException("Interrupted while connecting zookeeper", ie); - } catch (TextFormat.ParseException pe) { + } catch (RuntimeException pe) { throw new ReplicationException.UnavailableException("Error parsing proto message", pe); } } @@ -320,12 +315,12 @@ private void tryMarkLedgerUnderreplicatedAsync(final String znode, final Collection missingReplicas, final List zkAcls, final CompletableFuture finalFuture) { - final UnderreplicatedLedgerFormat.Builder builder = UnderreplicatedLedgerFormat.newBuilder(); + final UnderreplicatedLedgerFormat fmt = new UnderreplicatedLedgerFormat(); if (conf.getStoreSystemTimeAsLedgerUnderreplicatedMarkTime()) { - builder.setCtime(System.currentTimeMillis()); + fmt.setCtime(System.currentTimeMillis()); } - missingReplicas.forEach(builder::addReplica); - final byte[] urLedgerData = builder.build().toString().getBytes(UTF_8); + missingReplicas.forEach(fmt::addReplica); + final byte[] urLedgerData = fmt.toTextFormat().getBytes(UTF_8); ZkUtils.asyncCreateFullPathOptimistic( zkc, znode, urLedgerData, zkAcls, CreateMode.PERSISTENT, (rc, path, ctx, name) -> { @@ -349,23 +344,22 @@ private void handleLedgerUnderreplicatedAlreadyMarked(final String znode, zkc.getData(znode, false, (getRc, getPath, getCtx, existingUrLedgerData, getStat) -> { if (Code.OK.intValue() == getRc) { // deserialize existing underreplicated ledger data - final UnderreplicatedLedgerFormat.Builder builder = UnderreplicatedLedgerFormat.newBuilder(); + final UnderreplicatedLedgerFormat fmt = new UnderreplicatedLedgerFormat(); try { - TextFormat.merge(new String(existingUrLedgerData, UTF_8), builder); - } catch (ParseException e) { + fmt.parseFromTextFormat(existingUrLedgerData); + } catch (RuntimeException e) { // corrupted metadata in zookeeper FutureUtils.completeExceptionally(finalFuture, new ReplicationException.UnavailableException( "Invalid underreplicated ledger data for ledger " + znode, e)); return; } - UnderreplicatedLedgerFormat existingUrLedgerFormat = builder.build(); boolean replicaAdded = false; for (String missingReplica : missingReplicas) { - if (existingUrLedgerFormat.getReplicaList().contains(missingReplica)) { + if (fmt.getReplicasList().contains(missingReplica)) { continue; } else { - builder.addReplica(missingReplica); + fmt.addReplica(missingReplica); replicaAdded = true; } } @@ -374,9 +368,9 @@ private void handleLedgerUnderreplicatedAlreadyMarked(final String znode, return; } if (conf.getStoreSystemTimeAsLedgerUnderreplicatedMarkTime()) { - builder.setCtime(System.currentTimeMillis()); + fmt.setCtime(System.currentTimeMillis()); } - final byte[] newUrLedgerData = builder.build().toString().getBytes(UTF_8); + final byte[] newUrLedgerData = fmt.toTextFormat().getBytes(UTF_8); zkc.setData(znode, newUrLedgerData, getStat.getVersion(), (setRc, setPath, setCtx, setStat) -> { if (Code.OK.intValue() == setRc) { FutureUtils.complete(finalFuture, null); @@ -917,9 +911,8 @@ public String getReplicationWorkerIdRereplicatingLedger(long ledgerId) String replicationWorkerId = null; try { byte[] lockData = zkc.getData(getUrLedgerLockZnode(urLockPath, ledgerId), false, null); - LockDataFormat.Builder lockDataBuilder = LockDataFormat.newBuilder(); - TextFormat.merge(new String(lockData, UTF_8), lockDataBuilder); - LockDataFormat lock = lockDataBuilder.build(); + LockDataFormat lock = new LockDataFormat(); + lock.parseFromTextFormat(lockData); replicationWorkerId = lock.getBookieId(); } catch (KeeperException.NoNodeException e) { // this is ok. @@ -931,7 +924,7 @@ public String getReplicationWorkerIdRereplicatingLedger(long ledgerId) log.error().exception(e).log("Got interrupted while getting ReplicationWorkerId rereplicating Ledger"); Thread.currentThread().interrupt(); throw new ReplicationException.UnavailableException("Interrupted while contacting zookeeper", e); - } catch (ParseException e) { + } catch (RuntimeException e) { log.error().exception(e).log("Error while parsing ZK data of lock"); throw new ReplicationException.UnavailableException("Error while parsing ZK data of lock", e); } @@ -943,9 +936,9 @@ public void setCheckAllLedgersCTime(long checkAllLedgersCTime) throws Unavailabl log.debug("setCheckAllLedgersCTime"); try { List zkAcls = ZkUtils.getACLs(conf); - CheckAllLedgersFormat.Builder builder = CheckAllLedgersFormat.newBuilder(); - builder.setCheckAllLedgersCTime(checkAllLedgersCTime); - byte[] checkAllLedgersFormatByteArray = builder.build().toByteArray(); + CheckAllLedgersFormat fmt = new CheckAllLedgersFormat(); + fmt.setCheckAllLedgersCTime(checkAllLedgersCTime); + byte[] checkAllLedgersFormatByteArray = fmt.toByteArray(); if (zkc.exists(checkAllLedgersCtimeZnode, false) != null) { zkc.setData(checkAllLedgersCtimeZnode, checkAllLedgersFormatByteArray, -1); } else { @@ -964,7 +957,8 @@ public long getCheckAllLedgersCTime() throws UnavailableException { log.debug("getCheckAllLedgersCTime"); try { byte[] data = zkc.getData(checkAllLedgersCtimeZnode, false, null); - CheckAllLedgersFormat checkAllLedgersFormat = CheckAllLedgersFormat.parseFrom(data); + CheckAllLedgersFormat checkAllLedgersFormat = new CheckAllLedgersFormat(); + checkAllLedgersFormat.parseFrom(data); return checkAllLedgersFormat.hasCheckAllLedgersCTime() ? checkAllLedgersFormat.getCheckAllLedgersCTime() : -1; } catch (KeeperException.NoNodeException ne) { @@ -975,7 +969,7 @@ public long getCheckAllLedgersCTime() throws UnavailableException { } catch (InterruptedException ie) { Thread.currentThread().interrupt(); throw new ReplicationException.UnavailableException("Interrupted while contacting zookeeper", ie); - } catch (InvalidProtocolBufferException ipbe) { + } catch (RuntimeException ipbe) { throw new ReplicationException.UnavailableException("Error while parsing ZK protobuf binary data", ipbe); } } @@ -985,9 +979,9 @@ public void setPlacementPolicyCheckCTime(long placementPolicyCheckCTime) throws log.debug("setPlacementPolicyCheckCTime"); try { List zkAcls = ZkUtils.getACLs(conf); - PlacementPolicyCheckFormat.Builder builder = PlacementPolicyCheckFormat.newBuilder(); - builder.setPlacementPolicyCheckCTime(placementPolicyCheckCTime); - byte[] placementPolicyCheckFormatByteArray = builder.build().toByteArray(); + PlacementPolicyCheckFormat fmt = new PlacementPolicyCheckFormat(); + fmt.setPlacementPolicyCheckCTime(placementPolicyCheckCTime); + byte[] placementPolicyCheckFormatByteArray = fmt.toByteArray(); if (zkc.exists(placementPolicyCheckCtimeZnode, false) != null) { zkc.setData(placementPolicyCheckCtimeZnode, placementPolicyCheckFormatByteArray, -1); } else { @@ -1007,7 +1001,8 @@ public long getPlacementPolicyCheckCTime() throws UnavailableException { log.debug("getPlacementPolicyCheckCTime"); try { byte[] data = zkc.getData(placementPolicyCheckCtimeZnode, false, null); - PlacementPolicyCheckFormat placementPolicyCheckFormat = PlacementPolicyCheckFormat.parseFrom(data); + PlacementPolicyCheckFormat placementPolicyCheckFormat = new PlacementPolicyCheckFormat(); + placementPolicyCheckFormat.parseFrom(data); return placementPolicyCheckFormat.hasPlacementPolicyCheckCTime() ? placementPolicyCheckFormat.getPlacementPolicyCheckCTime() : -1; } catch (KeeperException.NoNodeException ne) { @@ -1018,7 +1013,7 @@ public long getPlacementPolicyCheckCTime() throws UnavailableException { } catch (InterruptedException ie) { Thread.currentThread().interrupt(); throw new ReplicationException.UnavailableException("Interrupted while contacting zookeeper", ie); - } catch (InvalidProtocolBufferException ipbe) { + } catch (RuntimeException ipbe) { throw new ReplicationException.UnavailableException("Error while parsing ZK protobuf binary data", ipbe); } } @@ -1027,9 +1022,9 @@ public long getPlacementPolicyCheckCTime() throws UnavailableException { public void setReplicasCheckCTime(long replicasCheckCTime) throws UnavailableException { try { List zkAcls = ZkUtils.getACLs(conf); - ReplicasCheckFormat.Builder builder = ReplicasCheckFormat.newBuilder(); - builder.setReplicasCheckCTime(replicasCheckCTime); - byte[] replicasCheckFormatByteArray = builder.build().toByteArray(); + ReplicasCheckFormat fmt = new ReplicasCheckFormat(); + fmt.setReplicasCheckCTime(replicasCheckCTime); + byte[] replicasCheckFormatByteArray = fmt.toByteArray(); if (zkc.exists(replicasCheckCtimeZnode, false) != null) { zkc.setData(replicasCheckCtimeZnode, replicasCheckFormatByteArray, -1); } else { @@ -1048,7 +1043,8 @@ public void setReplicasCheckCTime(long replicasCheckCTime) throws UnavailableExc public long getReplicasCheckCTime() throws UnavailableException { try { byte[] data = zkc.getData(replicasCheckCtimeZnode, false, null); - ReplicasCheckFormat replicasCheckFormat = ReplicasCheckFormat.parseFrom(data); + ReplicasCheckFormat replicasCheckFormat = new ReplicasCheckFormat(); + replicasCheckFormat.parseFrom(data); log.debug("getReplicasCheckCTime completed successfully"); return replicasCheckFormat.hasReplicasCheckCTime() ? replicasCheckFormat.getReplicasCheckCTime() : -1; } catch (KeeperException.NoNodeException ne) { @@ -1059,7 +1055,7 @@ public long getReplicasCheckCTime() throws UnavailableException { } catch (InterruptedException ie) { Thread.currentThread().interrupt(); throw new ReplicationException.UnavailableException("Interrupted while contacting zookeeper", ie); - } catch (InvalidProtocolBufferException ipbe) { + } catch (RuntimeException ipbe) { throw new ReplicationException.UnavailableException("Error while parsing ZK protobuf binary data", ipbe); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java index 114df52de83..b84492514a7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java @@ -31,7 +31,7 @@ import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.proto.BookieProtoEncoding; import org.apache.bookkeeper.proto.BookieProtocol; -import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.DigestType; +import org.apache.bookkeeper.proto.LedgerMetadataFormat.DigestType; import org.apache.bookkeeper.util.ByteBufList; import org.apache.bookkeeper.util.ByteBufVisitor; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java index 7269265d5a9..49e9645a2f7 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java @@ -95,7 +95,7 @@ import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieServer; -import org.apache.bookkeeper.proto.DataFormats.BookieServiceInfoFormat; +import org.apache.bookkeeper.proto.BookieServiceInfoFormat; import org.apache.bookkeeper.replication.AutoRecoveryMain; import org.apache.bookkeeper.replication.ReplicationStats; import org.apache.bookkeeper.server.Main; @@ -550,8 +550,9 @@ public void testBookieRegistrationBookieServiceInfo() throws Exception { byte[] bkRegNodeData = zkc.getData(bkRegPath, null, null); assertFalse("Bookie service info not written", bkRegNodeData == null || bkRegNodeData.length == 0); - BookieServiceInfoFormat serializedBookieServiceInfo = BookieServiceInfoFormat.parseFrom(bkRegNodeData); - BookieServiceInfoFormat.Endpoint serializedEndpoint = serializedBookieServiceInfo.getEndpoints(0); + BookieServiceInfoFormat serializedBookieServiceInfo = new BookieServiceInfoFormat(); + serializedBookieServiceInfo.parseFrom(bkRegNodeData); + BookieServiceInfoFormat.Endpoint serializedEndpoint = serializedBookieServiceInfo.getEndpointAt(0); assertNotNull("Serialized Bookie endpoint not found", serializedEndpoint); assertEquals(endpoint.getId(), serializedEndpoint.getId()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java index 3f8af53c133..8c664011589 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ClientUtil.java @@ -26,7 +26,7 @@ import java.util.function.Function; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.meta.LedgerManager; -import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.DigestType; +import org.apache.bookkeeper.proto.LedgerMetadataFormat.DigestType; import org.apache.bookkeeper.proto.MockBookieClient; import org.apache.bookkeeper.proto.checksum.DigestManager; import org.apache.bookkeeper.versioning.Versioned; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedOpTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedOpTest.java index 954bdccca83..e51ffd3ff7c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedOpTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedOpTest.java @@ -27,7 +27,7 @@ import org.apache.bookkeeper.common.util.OrderedExecutor; import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.DigestType; +import org.apache.bookkeeper.proto.LedgerMetadataFormat.DigestType; import org.apache.bookkeeper.proto.MockBookieClient; import org.apache.bookkeeper.proto.checksum.DigestManager; import org.junit.After; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookies.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookies.java index ac338b9757d..3b06e9c3edd 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookies.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookies.java @@ -32,6 +32,7 @@ import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.checksum.DigestManager; +import org.apache.bookkeeper.proto.LedgerMetadataFormat; import org.apache.bookkeeper.util.ByteBufList; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -82,7 +83,7 @@ public void seedEntries(BookieId bookieId, long ledgerId, long entryId, long lac public ByteBuf generateEntry(long ledgerId, long entryId, long lac) throws Exception { DigestManager digestManager = DigestManager.instantiate(ledgerId, new byte[0], - DataFormats.LedgerMetadataFormat.DigestType.CRC32C, + LedgerMetadataFormat.DigestType.CRC32C, UnpooledByteBufAllocator.DEFAULT, false); return ByteBufList.coalesce((ByteBufList) digestManager.computeDigestAndPackageForSending( entryId, lac, 0, Unpooled.buffer(10), new byte[20], 0)); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java index 3e418226e61..67d68b33539 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestLedgerUnderreplicationManager.java @@ -26,7 +26,6 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import com.google.protobuf.TextFormat; import java.nio.charset.Charset; import java.util.ArrayList; import java.util.Collection; @@ -53,7 +52,7 @@ import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.DNS; -import org.apache.bookkeeper.proto.DataFormats.UnderreplicatedLedgerFormat; +import org.apache.bookkeeper.proto.UnderreplicatedLedgerFormat; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.test.ZooKeeperUtil; import org.apache.bookkeeper.util.BookKeeperConstants; @@ -430,12 +429,11 @@ public void test2reportSame() throws Exception { m2.markLedgerUnderreplicated(ledgerA, missingReplica1); // verify duplicate missing replica - UnderreplicatedLedgerFormat.Builder builderA = UnderreplicatedLedgerFormat - .newBuilder(); + UnderreplicatedLedgerFormat fmtA = new UnderreplicatedLedgerFormat(); String znode = getUrLedgerZnode(ledgerA); byte[] data = zkc1.getData(znode, false, null); - TextFormat.merge(new String(data, Charset.forName("UTF-8")), builderA); - List replicaList = builderA.getReplicaList(); + fmtA.parseFromTextFormat(new String(data, Charset.forName("UTF-8"))); + List replicaList = fmtA.getReplicasList(); assertEquals("Published duplicate missing replica : " + replicaList, 1, replicaList.size()); assertTrue("Published duplicate missing replica : " + replicaList, @@ -833,12 +831,11 @@ private void verifyMarkLedgerUnderreplicated(Collection missingReplica) } String urLedgerA = getData(znodeA); - UnderreplicatedLedgerFormat.Builder builderA = UnderreplicatedLedgerFormat - .newBuilder(); + UnderreplicatedLedgerFormat fmtA = new UnderreplicatedLedgerFormat(); for (String replica : missingReplica) { - builderA.addReplica(replica); + fmtA.addReplica(replica); } - List replicaList = builderA.getReplicaList(); + List replicaList = fmtA.getReplicasList(); for (String replica : missingReplica) { assertTrue("UrLedger:" + urLedgerA diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java index d8ab2f5989a..e9928a01c86 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java @@ -70,7 +70,7 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.proto.BookkeeperProtocol; -import org.apache.bookkeeper.proto.DataFormats; +import org.apache.bookkeeper.proto.LedgerMetadataFormat; import org.apache.bookkeeper.proto.PerChannelBookieClient; import org.apache.bookkeeper.proto.PerChannelBookieClientPool; import org.apache.bookkeeper.proto.checksum.DigestManager; @@ -381,7 +381,7 @@ public void testBatchRead() throws Exception { byte[] passwd = new byte[20]; Arrays.fill(passwd, (byte) 'a'); DigestManager digestManager = DigestManager.instantiate(1, passwd, - DataFormats.LedgerMetadataFormat.DigestType.CRC32C, ByteBufAllocator.DEFAULT, true); + LedgerMetadataFormat.DigestType.CRC32C, ByteBufAllocator.DEFAULT, true); byte[] masterKey = DigestManager.generateMasterKey(passwd); final int entries = 10; @@ -443,7 +443,7 @@ public void testBatchedReadWittLostFourthEntry() throws Exception { byte[] passwd = new byte[20]; Arrays.fill(passwd, (byte) 'a'); DigestManager digestManager = DigestManager.instantiate(1, passwd, - DataFormats.LedgerMetadataFormat.DigestType.CRC32C, ByteBufAllocator.DEFAULT, true); + LedgerMetadataFormat.DigestType.CRC32C, ByteBufAllocator.DEFAULT, true); byte[] masterKey = DigestManager.generateMasterKey(passwd); final int entries = 10; @@ -509,7 +509,7 @@ public void testBatchedReadWittLostFirstEntry() throws Exception { byte[] passwd = new byte[20]; Arrays.fill(passwd, (byte) 'a'); DigestManager digestManager = DigestManager.instantiate(1, passwd, - DataFormats.LedgerMetadataFormat.DigestType.CRC32C, ByteBufAllocator.DEFAULT, true); + LedgerMetadataFormat.DigestType.CRC32C, ByteBufAllocator.DEFAULT, true); byte[] masterKey = DigestManager.generateMasterKey(passwd); final int entries = 10; @@ -560,7 +560,7 @@ public void testBatchedReadWittBigPayload() throws Exception { byte[] passwd = new byte[20]; Arrays.fill(passwd, (byte) 'a'); DigestManager digestManager = DigestManager.instantiate(1, passwd, - DataFormats.LedgerMetadataFormat.DigestType.CRC32C, ByteBufAllocator.DEFAULT, true); + LedgerMetadataFormat.DigestType.CRC32C, ByteBufAllocator.DEFAULT, true); byte[] masterKey = DigestManager.generateMasterKey(passwd); byte[] kbData = new byte[1024]; for (int i = 0; i < 1024; i++) { @@ -627,7 +627,7 @@ public void testBatchedReadWithMaxSizeLimitCase1() throws Exception { byte[] passwd = new byte[20]; Arrays.fill(passwd, (byte) 'a'); DigestManager digestManager = DigestManager.instantiate(1, passwd, - DataFormats.LedgerMetadataFormat.DigestType.CRC32C, ByteBufAllocator.DEFAULT, true); + LedgerMetadataFormat.DigestType.CRC32C, ByteBufAllocator.DEFAULT, true); byte[] masterKey = DigestManager.generateMasterKey(passwd); byte[] kbData = new byte[1024]; for (int i = 0; i < 1024; i++) { @@ -697,7 +697,7 @@ public void testBatchedReadWithMaxSizeLimitCase2() throws Exception { byte[] passwd = new byte[20]; Arrays.fill(passwd, (byte) 'a'); DigestManager digestManager = DigestManager.instantiate(1, passwd, - DataFormats.LedgerMetadataFormat.DigestType.CRC32C, ByteBufAllocator.DEFAULT, true); + LedgerMetadataFormat.DigestType.CRC32C, ByteBufAllocator.DEFAULT, true); byte[] masterKey = DigestManager.generateMasterKey(passwd); byte[] kbData = new byte[1024]; for (int i = 0; i < 1024; i++) { diff --git a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManagerBenchmark.java b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManagerBenchmark.java index cdf8f5a1ed7..5a7fede9264 100644 --- a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManagerBenchmark.java +++ b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManagerBenchmark.java @@ -25,7 +25,7 @@ import java.nio.charset.StandardCharsets; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; -import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.DigestType; +import org.apache.bookkeeper.proto.LedgerMetadataFormat.DigestType; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestTypeBenchmark.java b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestTypeBenchmark.java index 7201aba8cf0..39cb2b60693 100644 --- a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestTypeBenchmark.java +++ b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/checksum/DigestTypeBenchmark.java @@ -27,7 +27,7 @@ import java.nio.charset.StandardCharsets; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; -import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.DigestType; +import org.apache.bookkeeper.proto.LedgerMetadataFormat.DigestType; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; diff --git a/pom.xml b/pom.xml index c7ae8ee8786..3a9f2a9af1d 100644 --- a/pom.xml +++ b/pom.xml @@ -220,6 +220,7 @@ 3.10.1 1.7.1 0.6.1 + 0.7.0 9.3 4.7.3.2 3.6.0 From 19647faf4a48ed33fec6e97984459492e7b8e350 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 7 May 2026 13:47:03 -0700 Subject: [PATCH 03/11] Add SpotBugs excludes for lightproto-generated classes 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). --- .../src/main/resources/bookkeeper/findbugsExclude.xml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/buildtools/src/main/resources/bookkeeper/findbugsExclude.xml b/buildtools/src/main/resources/bookkeeper/findbugsExclude.xml index ea2b65e4474..326cfb36eea 100644 --- a/buildtools/src/main/resources/bookkeeper/findbugsExclude.xml +++ b/buildtools/src/main/resources/bookkeeper/findbugsExclude.xml @@ -40,15 +40,15 @@ - + + - - + + - - + From ff1f5532d3c427caf57945e6d7076dfa24f9c12c Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 7 May 2026 15:11:23 -0700 Subject: [PATCH 04/11] Fix checkstyle: remove redundant same-package import in MockBookies --- .../src/test/java/org/apache/bookkeeper/proto/MockBookies.java | 1 - 1 file changed, 1 deletion(-) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookies.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookies.java index 3b06e9c3edd..0cb3ad954fb 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookies.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookies.java @@ -32,7 +32,6 @@ import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.checksum.DigestManager; -import org.apache.bookkeeper.proto.LedgerMetadataFormat; import org.apache.bookkeeper.util.ByteBufList; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From 85ae81bc0edd0a923bf806db0aa6fc1f76bbdf93 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 7 May 2026 15:05:32 -0700 Subject: [PATCH 05/11] Migrate BookkeeperProtocol from protobuf-java to LightProto 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. --- bookkeeper-proto/pom.xml | 28 --- .../bookkeeper/client/BookieInfoReader.java | 14 +- .../bookkeeper/proto/AddCompletion.java | 10 +- .../apache/bookkeeper/proto/AuthHandler.java | 102 +++++----- .../proto/BatchedReadCompletion.java | 6 +- .../apache/bookkeeper/proto/BookieClient.java | 2 +- .../bookkeeper/proto/BookieClientImpl.java | 5 +- .../bookkeeper/proto/BookieNettyServer.java | 10 +- .../bookkeeper/proto/BookieProtoEncoding.java | 130 ++++-------- .../bookkeeper/proto/BookieProtocol.java | 2 +- .../proto/BookieRequestHandler.java | 2 +- .../proto/BookieRequestProcessor.java | 137 ++++++------- .../bookkeeper/proto/ByteStringUtil.java | 80 -------- .../bookkeeper/proto/CompletionKey.java | 2 +- .../bookkeeper/proto/CompletionValue.java | 10 +- .../bookkeeper/proto/EntryCompletionKey.java | 4 +- .../proto/ForceLedgerCompletion.java | 6 +- .../proto/ForceLedgerProcessorV3.java | 34 ++-- .../proto/GetBookieInfoCompletion.java | 6 +- .../proto/GetBookieInfoProcessorV3.java | 27 ++- .../GetListOfEntriesOfLedgerCompletion.java | 12 +- .../GetListOfEntriesOfLedgerProcessorV3.java | 30 ++- .../proto/LongPollReadEntryProcessorV3.java | 12 +- .../proto/PacketProcessorBaseV3.java | 17 +- .../proto/PerChannelBookieClient.java | 192 ++++++++---------- .../bookkeeper/proto/ReadCompletion.java | 12 +- .../proto/ReadEntryProcessorV3.java | 63 +++--- .../bookkeeper/proto/ReadLacCompletion.java | 10 +- .../bookkeeper/proto/ReadLacProcessorV3.java | 32 +-- .../apache/bookkeeper/proto/RequestUtils.java | 30 +-- .../bookkeeper/proto/StartTLSCompletion.java | 6 +- .../bookkeeper/proto/TxnCompletionKey.java | 2 +- .../proto/WriteEntryProcessorV3.java | 41 ++-- .../bookkeeper/proto/WriteLacCompletion.java | 6 +- .../bookkeeper/proto/WriteLacProcessorV3.java | 45 ++-- .../bookkeeper/util/StringEntryFormatter.java | 4 +- .../apache/bookkeeper/util/StringUtils.java | 7 +- .../client/TestGetBookieInfoTimeout.java | 10 +- ...TestLedgerFragmentReplicationWithMock.java | 2 +- .../proto/BookieProtoEncodingTest.java | 62 ++---- .../proto/ForceLedgerProcessorV3Test.java | 26 +-- .../proto/GetBookieInfoProcessorV3Test.java | 19 +- .../LongPollReadEntryProcessorV3Test.java | 36 ++-- .../proto/TestBackwardCompatCMS42.java | 36 ++-- .../proto/TestBookieRequestProcessor.java | 130 ++++++------ .../proto/TestPerChannelBookieClient.java | 14 +- .../proto/WriteEntryProcessorV3Test.java | 36 ++-- .../bookkeeper/test/BookieClientTest.java | 10 +- .../resources/bookkeeper/findbugsExclude.xml | 9 +- .../bookkeeper/proto/ProtocolBenchmark.java | 150 ++++++-------- 50 files changed, 693 insertions(+), 985 deletions(-) delete mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ByteStringUtil.java diff --git a/bookkeeper-proto/pom.xml b/bookkeeper-proto/pom.xml index d997fbde438..20b1d28a6ea 100644 --- a/bookkeeper-proto/pom.xml +++ b/bookkeeper-proto/pom.xml @@ -25,10 +25,6 @@ bookkeeper-proto Apache BookKeeper :: Protocols - - com.google.protobuf - protobuf-java - io.netty netty-buffer @@ -42,40 +38,16 @@ - **/BookkeeperProtocol.java target/generated-sources/lightproto/** **/.checkstyle - - org.xolstice.maven.plugins - protobuf-maven-plugin - ${protobuf-maven-plugin.version} - - com.google.protobuf:protoc:${protoc.version}:exe:${os.detected.classifier} - true - - BookkeeperProtocol.proto - - - - - - compile - - - - io.streamnative.lightproto lightproto-maven-plugin ${lightproto-maven-plugin.version} - - ${project.basedir}/src/main/proto/DataFormats.proto - ${project.basedir}/src/main/proto/DbLedgerStorageDataFormats.proto - generated-sources/lightproto/java true diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieInfoReader.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieInfoReader.java index b0f9636534c..669d453c55a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieInfoReader.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieInfoReader.java @@ -35,7 +35,7 @@ import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GetBookieInfoCallback; -import org.apache.bookkeeper.proto.BookkeeperProtocol; +import org.apache.bookkeeper.proto.GetBookieInfoRequest; import org.apache.commons.collections4.CollectionUtils; /** @@ -46,8 +46,8 @@ @CustomLog public class BookieInfoReader { private static final long GET_BOOKIE_INFO_REQUEST_FLAGS = - BookkeeperProtocol.GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE - | BookkeeperProtocol.GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE; + GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE + | GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE; private final ScheduledExecutorService scheduler; private final BookKeeper bk; @@ -329,8 +329,8 @@ synchronized void getReadWriteBookieInfo() { } BookieClient bkc = bk.getBookieClient(); - final long requested = BookkeeperProtocol.GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE - | BookkeeperProtocol.GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE; + final long requested = GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE + | GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE; totalSent = 0; completedCnt = 0; errorCnt = 0; @@ -413,8 +413,8 @@ Map getBookieInfo() throws BKException, InterruptedExcepti final ConcurrentMap map = new ConcurrentHashMap(); final CountDownLatch latch = new CountDownLatch(1); - long requested = BookkeeperProtocol.GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE - | BookkeeperProtocol.GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE; + long requested = GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE + | GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE; Collection bookies; bookies = bk.bookieWatcher.getBookies(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AddCompletion.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AddCompletion.java index 69409138b0c..83f99235eb6 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AddCompletion.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AddCompletion.java @@ -117,7 +117,7 @@ public void setOutstanding() { @Override public void handleV2Response( - long ledgerId, long entryId, BookkeeperProtocol.StatusCode status, + long ledgerId, long entryId, StatusCode status, BookieProtocol.Response response) { perChannelBookieClient.addEntryOutstanding.dec(); handleResponse(ledgerId, entryId, status); @@ -125,17 +125,17 @@ public void handleV2Response( @Override public void handleV3Response( - BookkeeperProtocol.Response response) { + Response response) { perChannelBookieClient.addEntryOutstanding.dec(); - BookkeeperProtocol.AddResponse addResponse = response.getAddResponse(); - BookkeeperProtocol.StatusCode status = response.getStatus() == BookkeeperProtocol.StatusCode.EOK + AddResponse addResponse = response.getAddResponse(); + StatusCode status = response.getStatus() == StatusCode.EOK ? addResponse.getStatus() : response.getStatus(); handleResponse(addResponse.getLedgerId(), addResponse.getEntryId(), status); } private void handleResponse(long ledgerId, long entryId, - BookkeeperProtocol.StatusCode status) { + StatusCode status) { logEvent(status).log("Got response from bookie"); int rc = convertStatus(status, BKException.Code.WriteException); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AuthHandler.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AuthHandler.java index ed2aa014b18..4f4f6c0ba0c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AuthHandler.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AuthHandler.java @@ -22,7 +22,6 @@ import static org.apache.bookkeeper.auth.AuthProviderFactoryFactory.AUTHENTICATION_DISABLED_PLUGIN_NAME; -import com.google.protobuf.ByteString; import io.netty.buffer.ByteBuf; import io.netty.channel.Channel; import io.netty.channel.ChannelDuplexHandler; @@ -40,7 +39,7 @@ import org.apache.bookkeeper.auth.BookieAuthProvider; import org.apache.bookkeeper.auth.ClientAuthProvider; import org.apache.bookkeeper.client.BKException; -import org.apache.bookkeeper.proto.BookkeeperProtocol.AuthMessage; +import org.apache.bookkeeper.proto.AuthMessage; import org.apache.bookkeeper.util.ByteBufList; import org.apache.bookkeeper.util.NettyChannelUtil; @@ -90,10 +89,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception BookieProtocol.AuthRequest req = (BookieProtocol.AuthRequest) msg; assert (req.getOpCode() == BookieProtocol.AUTH); if (checkAuthPlugin(req.getAuthMessage(), ctx.channel())) { - byte[] payload = req - .getAuthMessage() - .getPayload() - .toByteArray(); + byte[] payload = req.getAuthMessage().getPayload(); authProvider.process(AuthToken.wrap(payload), new AuthResponseCallbackLegacy(req, ctx.channel())); } else { @@ -114,26 +110,23 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception } else { ctx.channel().close(); } - } else if (msg instanceof BookkeeperProtocol.Request) { // post-PB-client - BookkeeperProtocol.Request req = (BookkeeperProtocol.Request) msg; - if (req.getHeader().getOperation() == BookkeeperProtocol.OperationType.AUTH + } else if (msg instanceof Request) { // post-PB-client + Request req = (Request) msg; + if (req.getHeader().getOperation() == OperationType.AUTH && req.hasAuthRequest() && checkAuthPlugin(req.getAuthRequest(), ctx.channel())) { - byte[] payload = req - .getAuthRequest() - .getPayload() - .toByteArray(); + byte[] payload = req.getAuthRequest().getPayload(); authProvider.process(AuthToken.wrap(payload), new AuthResponseCallback(req, ctx.channel(), authProviderFactory.getPluginName())); - } else if (req.getHeader().getOperation() == BookkeeperProtocol.OperationType.START_TLS + } else if (req.getHeader().getOperation() == OperationType.START_TLS && req.hasStartTLSRequest()) { super.channelRead(ctx, msg); } else { - BookkeeperProtocol.Response.Builder builder = BookkeeperProtocol.Response.newBuilder() - .setHeader(req.getHeader()) - .setStatus(BookkeeperProtocol.StatusCode.EUA); + Response response = new Response(); + response.setHeader().copyFrom(req.getHeader()); + response.setStatus(StatusCode.EUA); - NettyChannelUtil.writeAndFlushWithVoidPromise(ctx.channel(), builder.build()); + NettyChannelUtil.writeAndFlushWithVoidPromise(ctx.channel(), response); } } else { // close the channel, junk coming over it @@ -171,8 +164,9 @@ public void operationComplete(int rc, AuthToken newam) { channel.close(); return; } - AuthMessage message = AuthMessage.newBuilder().setAuthPluginName(req.authMessage.getAuthPluginName()) - .setPayload(ByteString.copyFrom(newam.getData())).build(); + AuthMessage message = new AuthMessage() + .setAuthPluginName(req.authMessage.getAuthPluginName()) + .setPayload(newam.getData()); final BookieProtocol.AuthResponse response = new BookieProtocol.AuthResponse(req.getProtocolVersion(), message); NettyChannelUtil.writeAndFlushWithVoidPromise(channel, response); @@ -180,11 +174,11 @@ public void operationComplete(int rc, AuthToken newam) { } static class AuthResponseCallback implements AuthCallbacks.GenericCallback { - final BookkeeperProtocol.Request req; + final Request req; final Channel channel; final String pluginName; - AuthResponseCallback(BookkeeperProtocol.Request req, Channel channel, String pluginName) { + AuthResponseCallback(Request req, Channel channel, String pluginName) { this.req = req; this.channel = channel; this.pluginName = pluginName; @@ -192,24 +186,21 @@ static class AuthResponseCallback implements AuthCallbacks.GenericCallback(); private final ClientAuthProvider.Factory authProviderFactory; - private final ExtensionRegistry registry; private final ClientConfiguration conf; private final ClientConfiguration v3Conf; @@ -116,7 +114,6 @@ public BookieClientImpl(ClientConfiguration conf, EventLoopGroup eventLoopGroup, this.closed = false; this.closeLock = new ReentrantReadWriteLock(); this.bookieAddressResolver = bookieAddressResolver; - this.registry = ExtensionRegistry.newInstance(); this.authProviderFactory = AuthProviderFactoryFactory.newClientAuthProviderFactory(conf); this.statsLogger = statsLogger; @@ -192,7 +189,7 @@ public PerChannelBookieClient create(BookieId address, PerChannelBookieClientPoo clientConfiguration = v3Conf; } return new PerChannelBookieClient(clientConfiguration, executor, eventLoopGroup, allocator, address, - statsLoggerForPCBC, authProviderFactory, registry, pcbcPool, + statsLoggerForPCBC, authProviderFactory, pcbcPool, shFactory, bookieAddressResolver); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java index bf1ca5168fb..cdb133e3fd8 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieNettyServer.java @@ -21,7 +21,6 @@ package org.apache.bookkeeper.proto; import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.ExtensionRegistry; import io.netty.bootstrap.ServerBootstrap; import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.PooledByteBufAllocator; @@ -109,7 +108,6 @@ class BookieNettyServer { final InetSocketAddress bindAddress; final BookieAuthProvider.Factory authProviderFactory; - final ExtensionRegistry registry = ExtensionRegistry.newInstance(); private final ByteBufAllocator allocator; @@ -382,8 +380,8 @@ protected void initChannel(SocketChannel ch) throws Exception { pipeline.addLast("lengthbaseddecoder", new LengthFieldBasedFrameDecoder(maxFrameSize, 0, 4, 0, 4)); - pipeline.addLast("bookieProtoDecoder", new BookieProtoEncoding.RequestDecoder(registry)); - pipeline.addLast("bookieProtoEncoder", new BookieProtoEncoding.ResponseEncoder(registry)); + pipeline.addLast("bookieProtoDecoder", new BookieProtoEncoding.RequestDecoder()); + pipeline.addLast("bookieProtoEncoder", new BookieProtoEncoding.ResponseEncoder()); pipeline.addLast("bookieAuthHandler", new AuthHandler.ServerSideHandler( contextHandler.getConnectionPeer(), authProviderFactory)); @@ -440,8 +438,8 @@ protected void initChannel(LocalChannel ch) throws Exception { pipeline.addLast("lengthbaseddecoder", new LengthFieldBasedFrameDecoder(maxFrameSize, 0, 4, 0, 4)); - pipeline.addLast("bookieProtoDecoder", new BookieProtoEncoding.RequestDecoder(registry)); - pipeline.addLast("bookieProtoEncoder", new BookieProtoEncoding.ResponseEncoder(registry)); + pipeline.addLast("bookieProtoDecoder", new BookieProtoEncoding.RequestDecoder()); + pipeline.addLast("bookieProtoEncoder", new BookieProtoEncoding.ResponseEncoder()); pipeline.addLast("bookieAuthHandler", new AuthHandler.ServerSideHandler( contextHandler.getConnectionPeer(), authProviderFactory)); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtoEncoding.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtoEncoding.java index 403813be6e6..374548aa825 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtoEncoding.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtoEncoding.java @@ -20,26 +20,17 @@ */ package org.apache.bookkeeper.proto; -import com.google.protobuf.CodedOutputStream; -import com.google.protobuf.ExtensionRegistry; -import com.google.protobuf.InvalidProtocolBufferException; -import com.google.protobuf.MessageLite; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; -import io.netty.buffer.ByteBufInputStream; -import io.netty.buffer.ByteBufOutputStream; import io.netty.channel.ChannelHandler.Sharable; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.channel.ChannelOutboundHandlerAdapter; import io.netty.channel.ChannelPromise; import io.netty.util.ReferenceCountUtil; -import java.io.IOException; import java.security.NoSuchAlgorithmException; import lombok.CustomLog; import org.apache.bookkeeper.proto.BookieProtocol.PacketHeader; -import org.apache.bookkeeper.proto.BookkeeperProtocol.OperationType; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Response; import org.apache.bookkeeper.proto.checksum.MacDigestManager; import org.apache.bookkeeper.util.ByteBufList; @@ -79,14 +70,12 @@ public interface EnDecoder { * @throws Exception */ Object decode(ByteBuf packet) throws Exception; - } /** * An encoder/decoder for the Bookkeeper protocol before version 3. */ public static class RequestEnDeCoderPreV3 implements EnDecoder { - final ExtensionRegistry extensionRegistry; //This empty master key is used when an empty password is provided which is the hash of an empty string private static final byte[] emptyPasswordMasterKey; @@ -98,10 +87,6 @@ public static class RequestEnDeCoderPreV3 implements EnDecoder { } } - public RequestEnDeCoderPreV3(ExtensionRegistry extensionRegistry) { - this.extensionRegistry = extensionRegistry; - } - @Override public Object encode(Object msg, ByteBufAllocator allocator) throws Exception { @@ -151,14 +136,13 @@ public Object encode(Object msg, ByteBufAllocator allocator) r.recycle(); return buf; } else if (r instanceof BookieProtocol.AuthRequest) { - BookkeeperProtocol.AuthMessage am = ((BookieProtocol.AuthRequest) r).getAuthMessage(); + AuthMessage am = ((BookieProtocol.AuthRequest) r).getAuthMessage(); int totalHeaderSize = 4; // for request type int totalSize = totalHeaderSize + am.getSerializedSize(); ByteBuf buf = allocator.buffer(totalSize + 4 /* frame size */); buf.writeInt(totalSize); buf.writeInt(PacketHeader.toInt(r.getProtocolVersion(), r.getOpCode(), r.getFlags())); - ByteBufOutputStream bufStream = new ByteBufOutputStream(buf); - am.writeTo(bufStream); + am.writeTo(buf); return buf; } else { return msg; @@ -219,9 +203,10 @@ public Object decode(ByteBuf packet) requestId, maxCount, maxSize); } case BookieProtocol.AUTH: - BookkeeperProtocol.AuthMessage.Builder builder = BookkeeperProtocol.AuthMessage.newBuilder(); - builder.mergeFrom(new ByteBufInputStream(packet), extensionRegistry); - return new BookieProtocol.AuthRequest(version, builder.build()); + AuthMessage am = new AuthMessage(); + am.parseFrom(packet, packet.readableBytes()); + am.materialize(); + return new BookieProtocol.AuthRequest(version, am); default: throw new IllegalStateException("Received unknown request op code = " + opCode); @@ -258,11 +243,6 @@ private static byte[] readMasterKey(ByteBuf packet) { * A response encoder/decoder for the Bookkeeper protocol before version 3. */ public static class ResponseEnDeCoderPreV3 implements EnDecoder { - final ExtensionRegistry extensionRegistry; - - public ResponseEnDeCoderPreV3(ExtensionRegistry extensionRegistry) { - this.extensionRegistry = extensionRegistry; - } private static final int RESPONSE_HEADERS_SIZE = 24; @@ -340,14 +320,14 @@ public Object encode(Object msg, ByteBufAllocator allocator) buf.writeLong(r.getEntryId()); return buf; } else if (msg instanceof BookieProtocol.AuthResponse) { - BookkeeperProtocol.AuthMessage am = ((BookieProtocol.AuthResponse) r).getAuthMessage(); + AuthMessage am = ((BookieProtocol.AuthResponse) r).getAuthMessage(); int payloadSize = 4 + am.getSerializedSize(); int bufferSize = payloadSize + 4 /* frame size */; ByteBuf buf = allocator.buffer(bufferSize); buf.writeInt(payloadSize); buf.writeInt(PacketHeader.toInt(r.getProtocolVersion(), r.getOpCode(), (short) 0)); - buf.writeBytes(am.toByteArray()); + am.writeTo(buf); return buf; } else { log.error().attr("type", msg.getClass().getName()).log("Cannot encode unknown response type"); @@ -400,10 +380,9 @@ public Object decode(ByteBuf buffer) return new BookieProtocol.BatchedReadResponse(version, rc, ledgerId, entryId, requestId, data == null ? ByteBufList.get() : data.retain()); case BookieProtocol.AUTH: - ByteBufInputStream bufStream = new ByteBufInputStream(buffer); - BookkeeperProtocol.AuthMessage.Builder builder = BookkeeperProtocol.AuthMessage.newBuilder(); - builder.mergeFrom(bufStream, extensionRegistry); - BookkeeperProtocol.AuthMessage am = builder.build(); + AuthMessage am = new AuthMessage(); + am.parseFrom(buffer, buffer.readableBytes()); + am.materialize(); return new BookieProtocol.AuthResponse(version, am); default: throw new IllegalStateException("Received unknown response : op code = " + opCode); @@ -423,21 +402,20 @@ public static void serializeAddResponseInto(int rc, BookieProtocol.ParsedAddRequ * A request encoder/decoder for the Bookkeeper protocol version 3. */ public static class RequestEnDecoderV3 implements EnDecoder { - final ExtensionRegistry extensionRegistry; - - public RequestEnDecoderV3(ExtensionRegistry extensionRegistry) { - this.extensionRegistry = extensionRegistry; - } @Override public Object decode(ByteBuf packet) throws Exception { - return BookkeeperProtocol.Request.parseFrom(new ByteBufInputStream(packet), extensionRegistry); + Request request = new Request(); + request.parseFrom(packet, packet.readableBytes()); + // Lightproto keeps references to the source buffer for lazy field access. + // Materialize so the request stays valid after the buffer is released. + request.materialize(); + return request; } @Override public Object encode(Object msg, ByteBufAllocator allocator) throws Exception { - BookkeeperProtocol.Request request = (BookkeeperProtocol.Request) msg; - return serializeProtobuf(request, allocator); + return serializeProtobuf((Request) msg, allocator); } } @@ -446,49 +424,33 @@ public Object encode(Object msg, ByteBufAllocator allocator) throws Exception { * A response encoder/decoder for the Bookkeeper protocol version 3. */ public static class ResponseEnDecoderV3 implements EnDecoder { - final ExtensionRegistry extensionRegistry; - - public ResponseEnDecoderV3(ExtensionRegistry extensionRegistry) { - this.extensionRegistry = extensionRegistry; - } @Override public Object decode(ByteBuf packet) throws Exception { - return BookkeeperProtocol.Response.parseFrom(new ByteBufInputStream(packet), - extensionRegistry); + Response response = new Response(); + response.parseFrom(packet, packet.readableBytes()); + // Lightproto keeps references to the source buffer for lazy field access. + // Materialize so the response stays valid after the buffer is released. + response.materialize(); + return response; } @Override public Object encode(Object msg, ByteBufAllocator allocator) throws Exception { - BookkeeperProtocol.Response response = (BookkeeperProtocol.Response) msg; - return serializeProtobuf(response, allocator); + return serializeProtobuf((Response) msg, allocator); } } - private static ByteBuf serializeProtobuf(MessageLite msg, ByteBufAllocator allocator) { + private static ByteBuf serializeProtobuf(LightProtoCodec.LightProtoMessage msg, ByteBufAllocator allocator) { int size = msg.getSerializedSize(); int frameSize = size + 4; - // Protobuf serialization is the last step of the netty pipeline. We used to allocate - // a heap buffer while serializing and pass it down to netty library. - // In AbstractChannel#filterOutboundMessage(), netty copies that data to a direct buffer if - // it is currently in heap (otherwise skips it and uses it directly). - // Allocating a direct buffer reducing unnecessary CPU cycles for buffer copies in BK client - // and also helps alleviate pressure off the GC, since there is less memory churn. - // Bookies aren't usually CPU bound. This change improves READ_ENTRY code paths by a small factor as well. + // Protobuf serialization is the last step of the netty pipeline. We allocate + // a direct buffer to avoid an extra copy in netty's AbstractChannel#filterOutboundMessage(). ByteBuf buf = allocator.directBuffer(frameSize, frameSize); buf.writeInt(size); - - try { - msg.writeTo(CodedOutputStream.newInstance(buf.nioBuffer(buf.writerIndex(), size))); - } catch (IOException e) { - // This is in-memory serialization, should not fail - throw new RuntimeException(e); - } - - // Advance writer idx - buf.writerIndex(frameSize); + msg.writeTo(buf); return buf; } @@ -501,9 +463,9 @@ public static class RequestEncoder extends ChannelOutboundHandlerAdapter { final EnDecoder reqPreV3; final EnDecoder reqV3; - public RequestEncoder(ExtensionRegistry extensionRegistry) { - reqPreV3 = new RequestEnDeCoderPreV3(extensionRegistry); - reqV3 = new RequestEnDecoderV3(extensionRegistry); + public RequestEncoder() { + reqPreV3 = new RequestEnDeCoderPreV3(); + reqV3 = new RequestEnDecoderV3(); } @Override @@ -514,7 +476,7 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) .log("Encode request to channel"); if (msg instanceof ByteBuf || msg instanceof ByteBufList) { ctx.write(msg, promise); - } else if (msg instanceof BookkeeperProtocol.Request) { + } else if (msg instanceof Request) { ctx.write(reqV3.encode(msg, ctx.alloc()), promise); } else if (msg instanceof BookieProtocol.Request) { ctx.write(reqPreV3.encode(msg, ctx.alloc()), promise); @@ -537,9 +499,9 @@ public static class RequestDecoder extends ChannelInboundHandlerAdapter { final EnDecoder reqV3; boolean usingV3Protocol; - RequestDecoder(ExtensionRegistry extensionRegistry) { - reqPreV3 = new RequestEnDeCoderPreV3(extensionRegistry); - reqV3 = new RequestEnDecoderV3(extensionRegistry); + public RequestDecoder() { + reqPreV3 = new RequestEnDeCoderPreV3(); + reqV3 = new RequestEnDecoderV3(); usingV3Protocol = true; } @@ -564,7 +526,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception if (usingV3Protocol) { try { result = reqV3.decode(buffer); - } catch (InvalidProtocolBufferException e) { + } catch (RuntimeException e) { usingV3Protocol = false; buffer.resetReaderIndex(); result = reqPreV3.decode(buffer); @@ -587,9 +549,9 @@ public static class ResponseEncoder extends ChannelOutboundHandlerAdapter { final EnDecoder repPreV3; final EnDecoder repV3; - ResponseEncoder(ExtensionRegistry extensionRegistry) { - repPreV3 = new ResponseEnDeCoderPreV3(extensionRegistry); - repV3 = new ResponseEnDecoderV3(extensionRegistry); + public ResponseEncoder() { + repPreV3 = new ResponseEnDeCoderPreV3(); + repV3 = new ResponseEnDecoderV3(); } @Override @@ -601,7 +563,7 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) if (msg instanceof ByteBuf) { ctx.write(msg, promise); - } else if (msg instanceof BookkeeperProtocol.Response) { + } else if (msg instanceof Response) { ctx.write(repV3.encode(msg, ctx.alloc()), promise); } else if (msg instanceof BookieProtocol.Response) { ctx.write(repPreV3.encode(msg, ctx.alloc()), promise); @@ -626,11 +588,9 @@ public static class ResponseDecoder extends ChannelInboundHandlerAdapter { final boolean tlsEnabled; boolean usingV3Protocol; - ResponseDecoder(ExtensionRegistry extensionRegistry, - boolean useV2Protocol, - boolean tlsEnabled) { - this.repPreV3 = new ResponseEnDeCoderPreV3(extensionRegistry); - this.repV3 = new ResponseEnDecoderV3(extensionRegistry); + public ResponseDecoder(boolean useV2Protocol, boolean tlsEnabled) { + this.repPreV3 = new ResponseEnDeCoderPreV3(); + this.repV3 = new ResponseEnDecoderV3(); this.useV2Protocol = useV2Protocol; this.tlsEnabled = tlsEnabled; usingV3Protocol = true; @@ -667,7 +627,7 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception usingV3Protocol = false; log.debug("Degrade bookkeeper to v2 after starting TLS."); } - } catch (InvalidProtocolBufferException e) { + } catch (RuntimeException e) { usingV3Protocol = false; buffer.resetReaderIndex(); result = repPreV3.decode(buffer); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtocol.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtocol.java index 6a93f8d2cc6..d202ccc8cc2 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtocol.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtocol.java @@ -26,7 +26,7 @@ import io.netty.util.Recycler.Handle; import io.netty.util.ReferenceCountUtil; import io.netty.util.ReferenceCounted; -import org.apache.bookkeeper.proto.BookkeeperProtocol.AuthMessage; +import org.apache.bookkeeper.proto.AuthMessage; import org.apache.bookkeeper.util.ByteBufList; /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestHandler.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestHandler.java index 999cca3a61d..90f0cd72d83 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestHandler.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestHandler.java @@ -89,7 +89,7 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws E @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { - if (!(msg instanceof BookkeeperProtocol.Request || msg instanceof BookieProtocol.Request)) { + if (!(msg instanceof Request || msg instanceof BookieProtocol.Request)) { ctx.fireChannelRead(msg); return; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestProcessor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestProcessor.java index 762f7220b43..2e1b0ab74fc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestProcessor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestProcessor.java @@ -27,7 +27,6 @@ import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; import com.google.common.util.concurrent.ThreadFactoryBuilder; -import com.google.protobuf.ByteString; import io.netty.buffer.ByteBufAllocator; import io.netty.channel.Channel; import io.netty.channel.group.ChannelGroup; @@ -316,11 +315,11 @@ public void processRequest(Object msg, BookieRequestHandler requestHandler) { Channel channel = requestHandler.ctx().channel(); // If we can decode this packet as a Request protobuf packet, process // it as a version 3 packet. Else, just use the old protocol. - if (msg instanceof BookkeeperProtocol.Request) { - BookkeeperProtocol.Request r = (BookkeeperProtocol.Request) msg; + if (msg instanceof Request) { + Request r = (Request) msg; restoreMdcContextFromRequest(r); try { - BookkeeperProtocol.BKPacketHeader header = r.getHeader(); + BKPacketHeader header = r.getHeader(); switch (header.getOperation()) { case ADD_ENTRY: processAddRequestV3(r, requestHandler); @@ -335,16 +334,12 @@ public void processRequest(Object msg, BookieRequestHandler requestHandler) { log.info() .attr("clientAddress", channel.remoteAddress()) .log("Ignoring auth operation from client"); - BookkeeperProtocol.AuthMessage message = BookkeeperProtocol.AuthMessage - .newBuilder() + Response authResponse = new Response(); + authResponse.setHeader().copyFrom(r.getHeader()); + authResponse.setStatus(StatusCode.EOK) + .setAuthResponse() .setAuthPluginName(AuthProviderFactoryFactory.AUTHENTICATION_DISABLED_PLUGIN_NAME) - .setPayload(ByteString.copyFrom(AuthToken.NULL.getData())) - .build(); - final BookkeeperProtocol.Response authResponse = BookkeeperProtocol.Response - .newBuilder().setHeader(r.getHeader()) - .setStatus(BookkeeperProtocol.StatusCode.EOK) - .setAuthResponse(message) - .build(); + .setPayload(AuthToken.NULL.getData()); writeAndFlush(channel, authResponse); break; case WRITE_LAC: @@ -364,10 +359,9 @@ public void processRequest(Object msg, BookieRequestHandler requestHandler) { break; default: log.info().attr("operationType", header.getOperation()).log("Unknown operation type"); - final BookkeeperProtocol.Response response = - BookkeeperProtocol.Response.newBuilder().setHeader(r.getHeader()) - .setStatus(BookkeeperProtocol.StatusCode.EBADREQ) - .build(); + Response response = new Response(); + response.setHeader().copyFrom(r.getHeader()); + response.setStatus(StatusCode.EBADREQ); writeAndFlush(channel, response); if (statsEnabled) { bkStats.getOpStats(BKStats.STATS_UNKNOWN).incrementFailedOps(); @@ -396,11 +390,9 @@ public void processRequest(Object msg, BookieRequestHandler requestHandler) { case BookieProtocol.AUTH: log.info().attr("clientAddress", requestHandler.ctx().channel().remoteAddress()) .log("Ignoring auth operation from client"); - BookkeeperProtocol.AuthMessage message = BookkeeperProtocol.AuthMessage - .newBuilder() + AuthMessage message = new AuthMessage() .setAuthPluginName(AuthProviderFactoryFactory.AUTHENTICATION_DISABLED_PLUGIN_NAME) - .setPayload(ByteString.copyFrom(AuthToken.NULL.getData())) - .build(); + .setPayload(AuthToken.NULL.getData()); final BookieProtocol.AuthResponse response = new BookieProtocol.AuthResponse( BookieProtocol.CURRENT_PROTOCOL_VERSION, message); @@ -419,36 +411,37 @@ public void processRequest(Object msg, BookieRequestHandler requestHandler) { } } - private void restoreMdcContextFromRequest(BookkeeperProtocol.Request req) { + private void restoreMdcContextFromRequest(Request req) { if (preserveMdcForTaskExecution) { MDC.clear(); - for (BookkeeperProtocol.ContextPair pair: req.getRequestContextList()) { + for (int i = 0; i < req.getRequestContextsCount(); i++) { + ContextPair pair = req.getRequestContextAt(i); MDC.put(pair.getKey(), pair.getValue()); } } } - private void processWriteLacRequestV3(final BookkeeperProtocol.Request r, + private void processWriteLacRequestV3(final Request r, final BookieRequestHandler requestHandler) { WriteLacProcessorV3 writeLac = new WriteLacProcessorV3(r, requestHandler, this); if (null == writeThreadPool) { writeLac.run(); } else { - writeThreadPool.executeOrdered(r.getAddRequest().getLedgerId(), writeLac); + writeThreadPool.executeOrdered(r.getWriteLacRequest().getLedgerId(), writeLac); } } - private void processReadLacRequestV3(final BookkeeperProtocol.Request r, + private void processReadLacRequestV3(final Request r, final BookieRequestHandler requestHandler) { ReadLacProcessorV3 readLac = new ReadLacProcessorV3(r, requestHandler, this); if (null == readThreadPool) { readLac.run(); } else { - readThreadPool.executeOrdered(r.getAddRequest().getLedgerId(), readLac); + readThreadPool.executeOrdered(r.getReadLacRequest().getLedgerId(), readLac); } } - private void processAddRequestV3(final BookkeeperProtocol.Request r, final BookieRequestHandler requestHandler) { + private void processAddRequestV3(final Request r, final BookieRequestHandler requestHandler) { WriteEntryProcessorV3 write = new WriteEntryProcessorV3(r, requestHandler, this); final OrderedExecutor threadPool; @@ -468,21 +461,19 @@ private void processAddRequestV3(final BookkeeperProtocol.Request r, final Booki .attr("entryId", () -> r.getAddRequest().getEntryId()) .log("Failed to process request to add entry. Too many pending requests"); getRequestStats().getAddEntryRejectedCounter().inc(); - BookkeeperProtocol.AddResponse.Builder addResponse = BookkeeperProtocol.AddResponse.newBuilder() + Response resp = new Response(); + resp.setHeader().copyFrom(write.getHeader()); + resp.setStatus(StatusCode.ETOOMANYREQUESTS); + resp.setAddResponse() .setLedgerId(r.getAddRequest().getLedgerId()) .setEntryId(r.getAddRequest().getEntryId()) - .setStatus(BookkeeperProtocol.StatusCode.ETOOMANYREQUESTS); - BookkeeperProtocol.Response.Builder response = BookkeeperProtocol.Response.newBuilder() - .setHeader(write.getHeader()) - .setStatus(addResponse.getStatus()) - .setAddResponse(addResponse); - BookkeeperProtocol.Response resp = response.build(); - write.sendResponse(addResponse.getStatus(), resp, requestStats.getAddRequestStats()); + .setStatus(StatusCode.ETOOMANYREQUESTS); + write.sendResponse(StatusCode.ETOOMANYREQUESTS, resp, requestStats.getAddRequestStats()); } } } - private void processForceLedgerRequestV3(final BookkeeperProtocol.Request r, + private void processForceLedgerRequestV3(final Request r, final BookieRequestHandler requestHandler) { ForceLedgerProcessorV3 forceLedger = new ForceLedgerProcessorV3(r, requestHandler, this); @@ -501,24 +492,21 @@ private void processForceLedgerRequestV3(final BookkeeperProtocol.Request r, } catch (RejectedExecutionException e) { log.debug().attr("ledgerId", () -> r.getForceLedgerRequest().getLedgerId()) .log("Failed to process request to force ledger. Too many pending requests"); - BookkeeperProtocol.ForceLedgerResponse.Builder forceLedgerResponse = - BookkeeperProtocol.ForceLedgerResponse.newBuilder() + Response resp = new Response(); + resp.setHeader().copyFrom(forceLedger.getHeader()); + resp.setStatus(StatusCode.ETOOMANYREQUESTS); + resp.setForceLedgerResponse() .setLedgerId(r.getForceLedgerRequest().getLedgerId()) - .setStatus(BookkeeperProtocol.StatusCode.ETOOMANYREQUESTS); - BookkeeperProtocol.Response.Builder response = BookkeeperProtocol.Response.newBuilder() - .setHeader(forceLedger.getHeader()) - .setStatus(forceLedgerResponse.getStatus()) - .setForceLedgerResponse(forceLedgerResponse); - BookkeeperProtocol.Response resp = response.build(); + .setStatus(StatusCode.ETOOMANYREQUESTS); forceLedger.sendResponse( - forceLedgerResponse.getStatus(), + StatusCode.ETOOMANYREQUESTS, resp, requestStats.getForceLedgerRequestStats()); } } } - private void processReadRequestV3(final BookkeeperProtocol.Request r, final BookieRequestHandler requestHandler) { + private void processReadRequestV3(final Request r, final BookieRequestHandler requestHandler) { ExecutorService fenceThread = null == highPriorityThreadPool ? null : highPriorityThreadPool.chooseThread(requestHandler.ctx()); @@ -537,7 +525,7 @@ private void processReadRequestV3(final BookkeeperProtocol.Request r, final Book // gets executed as fast as possible, so bypass the normal readThreadPool // and execute in highPriorityThreadPool boolean isHighPriority = RequestUtils.isHighPriority(r) - || hasFlag(r.getReadRequest(), BookkeeperProtocol.ReadRequest.Flag.FENCE_LEDGER); + || hasFlag(r.getReadRequest(), ReadRequest.Flag.FENCE_LEDGER); if (isHighPriority) { threadPool = highPriorityThreadPool; } else { @@ -555,35 +543,32 @@ private void processReadRequestV3(final BookkeeperProtocol.Request r, final Book .attr("entryId", () -> r.getReadRequest().getEntryId()) .log("Failed to process request to read entry. Too many pending requests"); getRequestStats().getReadEntryRejectedCounter().inc(); - BookkeeperProtocol.ReadResponse.Builder readResponse = BookkeeperProtocol.ReadResponse.newBuilder() - .setLedgerId(r.getReadRequest().getLedgerId()) - .setEntryId(r.getReadRequest().getEntryId()) - .setStatus(BookkeeperProtocol.StatusCode.ETOOMANYREQUESTS); - BookkeeperProtocol.Response.Builder response = BookkeeperProtocol.Response.newBuilder() - .setHeader(read.getHeader()) - .setStatus(readResponse.getStatus()) - .setReadResponse(readResponse); - BookkeeperProtocol.Response resp = response.build(); - read.sendResponse(readResponse.getStatus(), resp, requestStats.getReadRequestStats()); + Response resp = new Response(); + resp.setHeader().copyFrom(read.getHeader()); + resp.setStatus(StatusCode.ETOOMANYREQUESTS); + resp.setReadResponse() + .setLedgerId(r.getReadRequest().getLedgerId()) + .setEntryId(r.getReadRequest().getEntryId()) + .setStatus(StatusCode.ETOOMANYREQUESTS); + read.sendResponse(StatusCode.ETOOMANYREQUESTS, resp, requestStats.getReadRequestStats()); onReadRequestFinish(); } } } - private void processStartTLSRequestV3(final BookkeeperProtocol.Request r, + private void processStartTLSRequestV3(final Request r, final BookieRequestHandler requestHandler) { - BookkeeperProtocol.Response.Builder response = BookkeeperProtocol.Response.newBuilder(); - BookkeeperProtocol.BKPacketHeader.Builder header = BookkeeperProtocol.BKPacketHeader.newBuilder(); - header.setVersion(BookkeeperProtocol.ProtocolVersion.VERSION_THREE); - header.setOperation(r.getHeader().getOperation()); - header.setTxnId(r.getHeader().getTxnId()); - response.setHeader(header.build()); + Response response = new Response(); + response.setHeader() + .setVersion(ProtocolVersion.VERSION_THREE) + .setOperation(r.getHeader().getOperation()) + .setTxnId(r.getHeader().getTxnId()); final Channel c = requestHandler.ctx().channel(); if (shFactory == null) { log.error("Got StartTLS request but TLS not configured"); - response.setStatus(BookkeeperProtocol.StatusCode.EBADREQ); - writeAndFlush(c, response.build()); + response.setStatus(StatusCode.EBADREQ); + writeAndFlush(c, response); } else { log.info().attr("channel", c).log("Starting TLS handshake with client"); // there is no need to execute in a different thread as this operation is light @@ -595,9 +580,8 @@ private void processStartTLSRequestV3(final BookkeeperProtocol.Request r, c.pipeline().addFirst(TLS_HANDLER_NAME, sslHandler); } - response.setStatus(BookkeeperProtocol.StatusCode.EOK); - BookkeeperProtocol.StartTLSResponse.Builder builder = BookkeeperProtocol.StartTLSResponse.newBuilder(); - response.setStartTLSResponse(builder.build()); + response.setStatus(StatusCode.EOK); + response.setStartTLSResponse(); sslHandler.handshakeFuture().addListener(new GenericFutureListener>() { @Override public void operationComplete(Future future) throws Exception { @@ -619,10 +603,9 @@ public void operationComplete(Future future) throws Exception { } else { log.error().exception(future.cause()).log("TLS Handshake failure"); } - final BookkeeperProtocol.Response errResponse = BookkeeperProtocol.Response.newBuilder() - .setHeader(r.getHeader()) - .setStatus(BookkeeperProtocol.StatusCode.EIO) - .build(); + Response errResponse = new Response(); + errResponse.setHeader().copyFrom(r.getHeader()); + errResponse.setStatus(StatusCode.EIO); writeAndFlush(c, errResponse); if (statsEnabled) { bkStats.getOpStats(BKStats.STATS_UNKNOWN).incrementFailedOps(); @@ -630,11 +613,11 @@ public void operationComplete(Future future) throws Exception { } } }); - writeAndFlush(c, response.build()); + writeAndFlush(c, response); } } - private void processGetBookieInfoRequestV3(final BookkeeperProtocol.Request r, + private void processGetBookieInfoRequestV3(final Request r, final BookieRequestHandler requestHandler) { GetBookieInfoProcessorV3 getBookieInfo = new GetBookieInfoProcessorV3(r, requestHandler, this); if (null == readThreadPool) { @@ -644,7 +627,7 @@ private void processGetBookieInfoRequestV3(final BookkeeperProtocol.Request r, } } - private void processGetListOfEntriesOfLedgerProcessorV3(final BookkeeperProtocol.Request r, + private void processGetListOfEntriesOfLedgerProcessorV3(final Request r, final BookieRequestHandler requestHandler) { GetListOfEntriesOfLedgerProcessorV3 getListOfEntriesOfLedger = new GetListOfEntriesOfLedgerProcessorV3(r, requestHandler, this); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ByteStringUtil.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ByteStringUtil.java deleted file mode 100644 index b26ac7b36ab..00000000000 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ByteStringUtil.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.bookkeeper.proto; - -import com.google.protobuf.ByteString; -import com.google.protobuf.UnsafeByteOperations; -import io.netty.buffer.ByteBuf; -import java.nio.ByteBuffer; -import org.apache.bookkeeper.util.ByteBufList; - -public class ByteStringUtil { - - /** - * Wrap the internal buffers of a ByteBufList into a single ByteString. - * The lifecycle of the wrapped ByteString is tied to the ByteBufList. - * - * @param bufList ByteBufList to wrap - * @return ByteString wrapping the internal buffers of the ByteBufList - */ - public static ByteString byteBufListToByteString(ByteBufList bufList) { - ByteString aggregated = null; - for (int i = 0; i < bufList.size(); i++) { - ByteBuf buffer = bufList.getBuffer(i); - if (buffer.readableBytes() > 0) { - aggregated = byteBufToByteString(aggregated, buffer); - } - } - return aggregated != null ? aggregated : ByteString.EMPTY; - } - - /** - * Wrap the internal buffers of a ByteBuf into a single ByteString. - * The lifecycle of the wrapped ByteString is tied to the ByteBuf. - * - * @param byteBuf ByteBuf to wrap - * @return ByteString wrapping the internal buffers of the ByteBuf - */ - public static ByteString byteBufToByteString(ByteBuf byteBuf) { - return byteBufToByteString(null, byteBuf); - } - - // internal method to aggregate a ByteBuf into a single aggregated ByteString - private static ByteString byteBufToByteString(ByteString aggregated, ByteBuf byteBuf) { - if (byteBuf.readableBytes() == 0) { - return ByteString.EMPTY; - } - if (byteBuf.nioBufferCount() > 1) { - for (ByteBuffer nioBuffer : byteBuf.nioBuffers()) { - ByteString piece = UnsafeByteOperations.unsafeWrap(nioBuffer); - aggregated = (aggregated == null) ? piece : aggregated.concat(piece); - } - } else { - ByteString piece; - if (byteBuf.hasArray()) { - piece = UnsafeByteOperations.unsafeWrap(byteBuf.array(), byteBuf.arrayOffset() + byteBuf.readerIndex(), - byteBuf.readableBytes()); - } else { - piece = UnsafeByteOperations.unsafeWrap(byteBuf.nioBuffer()); - } - aggregated = (aggregated == null) ? piece : aggregated.concat(piece); - } - return aggregated; - } -} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/CompletionKey.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/CompletionKey.java index 0d543676e54..25c6be5ace7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/CompletionKey.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/CompletionKey.java @@ -21,7 +21,7 @@ package org.apache.bookkeeper.proto; -import org.apache.bookkeeper.proto.BookkeeperProtocol.OperationType; +import org.apache.bookkeeper.proto.OperationType; abstract class CompletionKey { OperationType operationType; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/CompletionValue.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/CompletionValue.java index 18bc7206bab..c01103fdcfb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/CompletionValue.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/CompletionValue.java @@ -89,7 +89,7 @@ void timeout() { TimeUnit.NANOSECONDS); errorOut(BKException.Code.TimeoutException); } - protected Event logEvent(BookkeeperProtocol.StatusCode status) { + protected Event logEvent(StatusCode status) { return log.debug() .attr("operation", operationName) .attr("bookieId", perChannelBookieClient.bookieId) @@ -98,7 +98,7 @@ protected Event logEvent(BookkeeperProtocol.StatusCode status) { .attr("status", status); } - protected int convertStatus(BookkeeperProtocol.StatusCode status, int defaultStatus) { + protected int convertStatus(StatusCode status, int defaultStatus) { // convert to BKException code int rcToRet = statusCodeToExceptionCode(status); if (rcToRet == BKException.Code.UNINITIALIZED) { @@ -119,7 +119,7 @@ protected int convertStatus(BookkeeperProtocol.StatusCode status, int defaultSta * @param status * @return {@link BKException.Code.UNINITIALIZED} if the statuscode is unknown. */ - private int statusCodeToExceptionCode(BookkeeperProtocol.StatusCode status) { + private int statusCodeToExceptionCode(StatusCode status) { switch (status) { case EOK: return BKException.Code.OK; @@ -171,13 +171,13 @@ protected void errorOutAndRunCallback(final Runnable callback) { } public void handleV2Response( - long ledgerId, long entryId, BookkeeperProtocol.StatusCode status, + long ledgerId, long entryId, StatusCode status, BookieProtocol.Response response) { log.warn().attr("response", response).log("Unhandled V2 response"); } public abstract void handleV3Response( - BookkeeperProtocol.Response response); + Response response); public void release() {} } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/EntryCompletionKey.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/EntryCompletionKey.java index 09f6413f7f3..f14fbeb2c98 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/EntryCompletionKey.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/EntryCompletionKey.java @@ -29,7 +29,7 @@ class EntryCompletionKey extends CompletionKey { long entryId; static EntryCompletionKey acquireV2Key(long ledgerId, long entryId, - BookkeeperProtocol.OperationType operationType) { + OperationType operationType) { EntryCompletionKey key = V2_KEY_RECYCLER.get(); key.reset(ledgerId, entryId, operationType); return key; @@ -40,7 +40,7 @@ private EntryCompletionKey(Recycler.Handle handle) { this.recyclerHandle = handle; } - void reset(long ledgerId, long entryId, BookkeeperProtocol.OperationType operationType) { + void reset(long ledgerId, long entryId, OperationType operationType) { this.ledgerId = ledgerId; this.entryId = entryId; this.operationType = operationType; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerCompletion.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerCompletion.java index 0e964c06589..6e52d087616 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerCompletion.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerCompletion.java @@ -55,9 +55,9 @@ public void errorOut(final int rc) { } @Override - public void handleV3Response(BookkeeperProtocol.Response response) { - BookkeeperProtocol.ForceLedgerResponse forceLedgerResponse = response.getForceLedgerResponse(); - BookkeeperProtocol.StatusCode status = response.getStatus() == BookkeeperProtocol.StatusCode.EOK + public void handleV3Response(Response response) { + ForceLedgerResponse forceLedgerResponse = response.getForceLedgerResponse(); + StatusCode status = response.getStatus() == StatusCode.EOK ? forceLedgerResponse.getStatus() : response.getStatus(); long ledgerId = forceLedgerResponse.getLedgerId(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java index 0c1a55e02ab..dd40d45f90e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java @@ -27,11 +27,11 @@ import org.apache.bookkeeper.bookie.BookieImpl; import org.apache.bookkeeper.common.util.MathUtils; import org.apache.bookkeeper.net.BookieId; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ForceLedgerRequest; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ForceLedgerResponse; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Response; -import org.apache.bookkeeper.proto.BookkeeperProtocol.StatusCode; +import org.apache.bookkeeper.proto.ForceLedgerRequest; +import org.apache.bookkeeper.proto.ForceLedgerResponse; +import org.apache.bookkeeper.proto.Request; +import org.apache.bookkeeper.proto.Response; +import org.apache.bookkeeper.proto.StatusCode; @CustomLog @@ -48,11 +48,11 @@ private ForceLedgerResponse getForceLedgerResponse() { ForceLedgerRequest forceLedgerRequest = request.getForceLedgerRequest(); long ledgerId = forceLedgerRequest.getLedgerId(); - final ForceLedgerResponse.Builder forceLedgerResponse = ForceLedgerResponse.newBuilder().setLedgerId(ledgerId); + final ForceLedgerResponse forceLedgerResponse = new ForceLedgerResponse().setLedgerId(ledgerId); if (!isVersionCompatible()) { forceLedgerResponse.setStatus(StatusCode.EBADVERSION); - return forceLedgerResponse.build(); + return forceLedgerResponse; } BookkeeperInternalCallbacks.WriteCallback wcb = @@ -87,11 +87,10 @@ private ForceLedgerResponse getForceLedgerResponse() { break; } forceLedgerResponse.setStatus(status); - Response.Builder response = Response.newBuilder() - .setHeader(getHeader()) - .setStatus(forceLedgerResponse.getStatus()) - .setForceLedgerResponse(forceLedgerResponse); - Response resp = response.build(); + Response resp = new Response(); + resp.setHeader().copyFrom(getHeader()); + resp.setStatus(forceLedgerResponse.getStatus()); + resp.setForceLedgerResponse().copyFrom(forceLedgerResponse); sendResponse(status, resp, requestProcessor.getRequestStats().getForceLedgerRequestStats()); }; StatusCode status = null; @@ -111,7 +110,7 @@ private ForceLedgerResponse getForceLedgerResponse() { // doesn't return a response back to the caller. if (!status.equals(StatusCode.EOK)) { forceLedgerResponse.setStatus(status); - return forceLedgerResponse.build(); + return forceLedgerResponse; } return null; } @@ -120,11 +119,10 @@ private ForceLedgerResponse getForceLedgerResponse() { public void run() { ForceLedgerResponse forceLedgerResponse = getForceLedgerResponse(); if (null != forceLedgerResponse) { - Response.Builder response = Response.newBuilder() - .setHeader(getHeader()) - .setStatus(forceLedgerResponse.getStatus()) - .setForceLedgerResponse(forceLedgerResponse); - Response resp = response.build(); + Response resp = new Response(); + resp.setHeader().copyFrom(getHeader()); + resp.setStatus(forceLedgerResponse.getStatus()); + resp.setForceLedgerResponse().copyFrom(forceLedgerResponse); sendResponse( forceLedgerResponse.getStatus(), resp, diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetBookieInfoCompletion.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetBookieInfoCompletion.java index 83feb969f1c..497bdf347e2 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetBookieInfoCompletion.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetBookieInfoCompletion.java @@ -64,9 +64,9 @@ public void errorOut(final int rc) { } @Override - public void handleV3Response(BookkeeperProtocol.Response response) { - BookkeeperProtocol.GetBookieInfoResponse getBookieInfoResponse = response.getGetBookieInfoResponse(); - BookkeeperProtocol.StatusCode status = response.getStatus() == BookkeeperProtocol.StatusCode.EOK + public void handleV3Response(Response response) { + GetBookieInfoResponse getBookieInfoResponse = response.getGetBookieInfoResponse(); + StatusCode status = response.getStatus() == StatusCode.EOK ? getBookieInfoResponse.getStatus() : response.getStatus(); long freeDiskSpace = getBookieInfoResponse.getFreeDiskSpace(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetBookieInfoProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetBookieInfoProcessorV3.java index 875a91fe012..ce051a27662 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetBookieInfoProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetBookieInfoProcessorV3.java @@ -24,11 +24,11 @@ import java.util.concurrent.TimeUnit; import lombok.CustomLog; import org.apache.bookkeeper.common.util.MathUtils; -import org.apache.bookkeeper.proto.BookkeeperProtocol.GetBookieInfoRequest; -import org.apache.bookkeeper.proto.BookkeeperProtocol.GetBookieInfoResponse; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Response; -import org.apache.bookkeeper.proto.BookkeeperProtocol.StatusCode; +import org.apache.bookkeeper.proto.GetBookieInfoRequest; +import org.apache.bookkeeper.proto.GetBookieInfoResponse; +import org.apache.bookkeeper.proto.Request; +import org.apache.bookkeeper.proto.Response; +import org.apache.bookkeeper.proto.StatusCode; /** * A processor class for v3 bookie metadata packets. @@ -46,13 +46,13 @@ private GetBookieInfoResponse getGetBookieInfoResponse() { GetBookieInfoRequest getBookieInfoRequest = request.getGetBookieInfoRequest(); long requested = getBookieInfoRequest.getRequested(); - GetBookieInfoResponse.Builder getBookieInfoResponse = GetBookieInfoResponse.newBuilder(); + GetBookieInfoResponse getBookieInfoResponse = new GetBookieInfoResponse(); if (!isVersionCompatible()) { getBookieInfoResponse.setStatus(StatusCode.EBADVERSION); requestProcessor.getRequestStats().getGetBookieInfoStats() .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); - return getBookieInfoResponse.build(); + return getBookieInfoResponse; } log.debug().attr("request", request).log("Received new getBookieInfo request"); @@ -81,7 +81,7 @@ private GetBookieInfoResponse getGetBookieInfoResponse() { } getBookieInfoResponse.setStatus(status); - return getBookieInfoResponse.build(); + return getBookieInfoResponse; } @Override @@ -91,12 +91,11 @@ public void run() { } private void sendResponse(GetBookieInfoResponse getBookieInfoResponse) { - Response.Builder response = Response.newBuilder() - .setHeader(getHeader()) - .setStatus(getBookieInfoResponse.getStatus()) - .setGetBookieInfoResponse(getBookieInfoResponse); - sendResponse(response.getStatus(), - response.build(), + Response response = new Response(); + response.setHeader().copyFrom(getHeader()); + response.setStatus(getBookieInfoResponse.getStatus()); + response.setGetBookieInfoResponse().copyFrom(getBookieInfoResponse); + sendResponse(response.getStatus(), response, requestProcessor.getRequestStats().getGetBookieInfoRequestStats()); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetListOfEntriesOfLedgerCompletion.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetListOfEntriesOfLedgerCompletion.java index d500604ae86..4b0cedaba05 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetListOfEntriesOfLedgerCompletion.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetListOfEntriesOfLedgerCompletion.java @@ -56,17 +56,17 @@ public void errorOut(final int rc) { } @Override - public void handleV3Response(BookkeeperProtocol.Response response) { - BookkeeperProtocol.GetListOfEntriesOfLedgerResponse getListOfEntriesOfLedgerResponse = response + public void handleV3Response(Response response) { + GetListOfEntriesOfLedgerResponse getListOfEntriesOfLedgerResponse = response .getGetListOfEntriesOfLedgerResponse(); ByteBuf availabilityOfEntriesOfLedgerBuffer = Unpooled.EMPTY_BUFFER; - BookkeeperProtocol.StatusCode status = - response.getStatus() == BookkeeperProtocol.StatusCode.EOK ? getListOfEntriesOfLedgerResponse.getStatus() + StatusCode status = + response.getStatus() == StatusCode.EOK ? getListOfEntriesOfLedgerResponse.getStatus() : response.getStatus(); if (getListOfEntriesOfLedgerResponse.hasAvailabilityOfEntriesOfLedger()) { - availabilityOfEntriesOfLedgerBuffer = Unpooled.wrappedBuffer( - getListOfEntriesOfLedgerResponse.getAvailabilityOfEntriesOfLedger().asReadOnlyByteBuffer()); + availabilityOfEntriesOfLedgerBuffer = + getListOfEntriesOfLedgerResponse.getAvailabilityOfEntriesOfLedgerSlice(); } logEvent(status).log("Got response from bookie"); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetListOfEntriesOfLedgerProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetListOfEntriesOfLedgerProcessorV3.java index b54a368f09c..7eef5935617 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetListOfEntriesOfLedgerProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetListOfEntriesOfLedgerProcessorV3.java @@ -20,17 +20,16 @@ */ package org.apache.bookkeeper.proto; -import com.google.protobuf.ByteString; import java.io.IOException; import java.util.concurrent.TimeUnit; import lombok.CustomLog; import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.common.util.MathUtils; -import org.apache.bookkeeper.proto.BookkeeperProtocol.GetListOfEntriesOfLedgerRequest; -import org.apache.bookkeeper.proto.BookkeeperProtocol.GetListOfEntriesOfLedgerResponse; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Response; -import org.apache.bookkeeper.proto.BookkeeperProtocol.StatusCode; +import org.apache.bookkeeper.proto.GetListOfEntriesOfLedgerRequest; +import org.apache.bookkeeper.proto.GetListOfEntriesOfLedgerResponse; +import org.apache.bookkeeper.proto.Request; +import org.apache.bookkeeper.proto.Response; +import org.apache.bookkeeper.proto.StatusCode; import org.apache.bookkeeper.util.AvailabilityOfEntriesOfLedger; /** @@ -51,15 +50,14 @@ public GetListOfEntriesOfLedgerProcessorV3(Request request, BookieRequestHandler private GetListOfEntriesOfLedgerResponse getListOfEntriesOfLedgerResponse() { long startTimeNanos = MathUtils.nowInNano(); - GetListOfEntriesOfLedgerResponse.Builder getListOfEntriesOfLedgerResponse = GetListOfEntriesOfLedgerResponse - .newBuilder(); + GetListOfEntriesOfLedgerResponse getListOfEntriesOfLedgerResponse = new GetListOfEntriesOfLedgerResponse(); getListOfEntriesOfLedgerResponse.setLedgerId(ledgerId); if (!isVersionCompatible()) { getListOfEntriesOfLedgerResponse.setStatus(StatusCode.EBADVERSION); requestProcessor.getRequestStats().getGetListOfEntriesOfLedgerStats() .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); - return getListOfEntriesOfLedgerResponse.build(); + return getListOfEntriesOfLedgerResponse; } log.debug().attr("request", request).log("Received new getListOfEntriesOfLedger request"); @@ -69,7 +67,7 @@ private GetListOfEntriesOfLedgerResponse getListOfEntriesOfLedgerResponse() { availabilityOfEntriesOfLedger = new AvailabilityOfEntriesOfLedger( requestProcessor.bookie.getListOfEntriesOfLedger(ledgerId)); getListOfEntriesOfLedgerResponse.setAvailabilityOfEntriesOfLedger( - ByteString.copyFrom(availabilityOfEntriesOfLedger.serializeStateOfEntriesOfLedger())); + availabilityOfEntriesOfLedger.serializeStateOfEntriesOfLedger()); } catch (Bookie.NoLedgerException e) { status = StatusCode.ENOLEDGER; @@ -93,17 +91,17 @@ private GetListOfEntriesOfLedgerResponse getListOfEntriesOfLedgerResponse() { } // Finally set the status and return getListOfEntriesOfLedgerResponse.setStatus(status); - return getListOfEntriesOfLedgerResponse.build(); + return getListOfEntriesOfLedgerResponse; } @Override public void run() { GetListOfEntriesOfLedgerResponse listOfEntriesOfLedgerResponse = getListOfEntriesOfLedgerResponse(); - Response.Builder response = Response.newBuilder().setHeader(getHeader()) - .setStatus(listOfEntriesOfLedgerResponse.getStatus()) - .setGetListOfEntriesOfLedgerResponse(listOfEntriesOfLedgerResponse); - Response resp = response.build(); - sendResponse(listOfEntriesOfLedgerResponse.getStatus(), resp, + Response response = new Response(); + response.setHeader().copyFrom(getHeader()); + response.setStatus(listOfEntriesOfLedgerResponse.getStatus()); + response.setGetListOfEntriesOfLedgerResponse().copyFrom(listOfEntriesOfLedgerResponse); + sendResponse(listOfEntriesOfLedgerResponse.getStatus(), response, requestProcessor.getRequestStats().getListOfEntriesOfLedgerRequestStats); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java index 67c82dda977..73d824c5b45 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java @@ -31,9 +31,9 @@ import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.LastAddConfirmedUpdateNotification; import org.apache.bookkeeper.common.util.Watcher; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ReadResponse; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; -import org.apache.bookkeeper.proto.BookkeeperProtocol.StatusCode; +import org.apache.bookkeeper.proto.ReadResponse; +import org.apache.bookkeeper.proto.Request; +import org.apache.bookkeeper.proto.StatusCode; /** * Processor handling long poll read entry request. @@ -74,7 +74,7 @@ private synchronized boolean shouldReadEntry() { } @Override - protected ReadResponse readEntry(ReadResponse.Builder readResponseBuilder, + protected ReadResponse readEntry(ReadResponse readResponseBuilder, long entryId, Stopwatch startTimeSw) throws IOException, BookieException { @@ -129,10 +129,10 @@ protected ReadResponse readEntry(ReadResponse.Builder readResponseBuilder, } private ReadResponse buildErrorResponse(StatusCode statusCode, Stopwatch sw) { - ReadResponse.Builder builder = ReadResponse.newBuilder() + ReadResponse readResponse = new ReadResponse() .setLedgerId(ledgerId) .setEntryId(entryId); - return buildResponse(builder, statusCode, sw); + return buildResponse(readResponse, statusCode, sw); } private ReadResponse getLongPollReadResponse() { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PacketProcessorBaseV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PacketProcessorBaseV3.java index b9f257ceca9..acff521cea4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PacketProcessorBaseV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PacketProcessorBaseV3.java @@ -26,10 +26,10 @@ import java.util.concurrent.TimeUnit; import lombok.CustomLog; import org.apache.bookkeeper.common.util.MathUtils; -import org.apache.bookkeeper.proto.BookkeeperProtocol.BKPacketHeader; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ProtocolVersion; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; -import org.apache.bookkeeper.proto.BookkeeperProtocol.StatusCode; +import org.apache.bookkeeper.proto.BKPacketHeader; +import org.apache.bookkeeper.proto.ProtocolVersion; +import org.apache.bookkeeper.proto.Request; +import org.apache.bookkeeper.proto.StatusCode; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.util.StringUtils; @@ -125,11 +125,10 @@ protected boolean isVersionCompatible() { * @return */ protected BKPacketHeader getHeader() { - BKPacketHeader.Builder header = BKPacketHeader.newBuilder(); - header.setVersion(ProtocolVersion.VERSION_THREE); - header.setOperation(request.getHeader().getOperation()); - header.setTxnId(request.getHeader().getTxnId()); - return header.build(); + return new BKPacketHeader() + .setVersion(ProtocolVersion.VERSION_THREE) + .setOperation(request.getHeader().getOperation()) + .setTxnId(request.getHeader().getTxnId()); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java index c68f7929646..b2a90187d43 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java @@ -19,9 +19,6 @@ package org.apache.bookkeeper.proto; import com.google.common.collect.Sets; -import com.google.protobuf.ByteString; -import com.google.protobuf.ExtensionRegistry; -import com.google.protobuf.UnsafeByteOperations; import io.github.merlimat.slog.Logger; import io.netty.bootstrap.Bootstrap; import io.netty.buffer.ByteBuf; @@ -106,19 +103,19 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadLacCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteLacCallback; -import org.apache.bookkeeper.proto.BookkeeperProtocol.AddRequest; -import org.apache.bookkeeper.proto.BookkeeperProtocol.BKPacketHeader; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ForceLedgerRequest; -import org.apache.bookkeeper.proto.BookkeeperProtocol.GetBookieInfoRequest; -import org.apache.bookkeeper.proto.BookkeeperProtocol.GetListOfEntriesOfLedgerRequest; -import org.apache.bookkeeper.proto.BookkeeperProtocol.OperationType; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ProtocolVersion; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ReadLacRequest; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ReadRequest; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Response; -import org.apache.bookkeeper.proto.BookkeeperProtocol.StatusCode; -import org.apache.bookkeeper.proto.BookkeeperProtocol.WriteLacRequest; +import org.apache.bookkeeper.proto.AddRequest; +import org.apache.bookkeeper.proto.BKPacketHeader; +import org.apache.bookkeeper.proto.ForceLedgerRequest; +import org.apache.bookkeeper.proto.GetBookieInfoRequest; +import org.apache.bookkeeper.proto.GetListOfEntriesOfLedgerRequest; +import org.apache.bookkeeper.proto.OperationType; +import org.apache.bookkeeper.proto.ProtocolVersion; +import org.apache.bookkeeper.proto.ReadLacRequest; +import org.apache.bookkeeper.proto.ReadRequest; +import org.apache.bookkeeper.proto.Request; +import org.apache.bookkeeper.proto.Response; +import org.apache.bookkeeper.proto.StatusCode; +import org.apache.bookkeeper.proto.WriteLacRequest; import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.OpStatsLogger; @@ -330,35 +327,33 @@ enum ConnectionState { private final PerChannelBookieClientPool pcbcPool; private final ClientAuthProvider.Factory authProviderFactory; - private final ExtensionRegistry extRegistry; private final SecurityHandlerFactory shFactory; private volatile boolean isWritable = true; private long lastBookieUnavailableLogTimestamp = 0; public PerChannelBookieClient(OrderedExecutor executor, EventLoopGroup eventLoopGroup, BookieId addr, BookieAddressResolver bookieAddressResolver) throws SecurityException { - this(new ClientConfiguration(), executor, eventLoopGroup, addr, NullStatsLogger.INSTANCE, null, null, + this(new ClientConfiguration(), executor, eventLoopGroup, addr, NullStatsLogger.INSTANCE, null, null, bookieAddressResolver); } public PerChannelBookieClient(OrderedExecutor executor, EventLoopGroup eventLoopGroup, BookieId bookieId, ClientAuthProvider.Factory authProviderFactory, - ExtensionRegistry extRegistry, BookieAddressResolver bookieAddressResolver) + BookieAddressResolver bookieAddressResolver) throws SecurityException { this(new ClientConfiguration(), executor, eventLoopGroup, bookieId, NullStatsLogger.INSTANCE, - authProviderFactory, extRegistry, null, bookieAddressResolver); + authProviderFactory, null, bookieAddressResolver); } public PerChannelBookieClient(ClientConfiguration conf, OrderedExecutor executor, EventLoopGroup eventLoopGroup, BookieId bookieId, StatsLogger parentStatsLogger, ClientAuthProvider.Factory authProviderFactory, - ExtensionRegistry extRegistry, PerChannelBookieClientPool pcbcPool, BookieAddressResolver bookieAddressResolver) throws SecurityException { this(conf, executor, eventLoopGroup, UnpooledByteBufAllocator.DEFAULT, bookieId, NullStatsLogger.INSTANCE, - authProviderFactory, extRegistry, pcbcPool, null, bookieAddressResolver); + authProviderFactory, pcbcPool, null, bookieAddressResolver); } public PerChannelBookieClient(ClientConfiguration conf, OrderedExecutor executor, @@ -366,7 +361,6 @@ public PerChannelBookieClient(ClientConfiguration conf, OrderedExecutor executor ByteBufAllocator allocator, BookieId bookieId, StatsLogger parentStatsLogger, ClientAuthProvider.Factory authProviderFactory, - ExtensionRegistry extRegistry, PerChannelBookieClientPool pcbcPool, SecurityHandlerFactory shFactory, BookieAddressResolver bookieAddressResolver) throws SecurityException { @@ -395,7 +389,6 @@ public PerChannelBookieClient(ClientConfiguration conf, OrderedExecutor executor this.preserveMdcForTaskExecution = conf.getPreserveMdcForTaskExecution(); this.authProviderFactory = authProviderFactory; - this.extRegistry = extRegistry; this.shFactory = shFactory; if (shFactory != null) { shFactory.init(NodeType.Client, conf, allocator); @@ -609,10 +602,10 @@ protected void initChannel(Channel ch) throws Exception { pipeline.addLast("bytebufList", ByteBufList.ENCODER); pipeline.addLast("lengthbasedframedecoder", new LengthFieldBasedFrameDecoder(maxFrameSize, 0, 4, 0, 4)); - pipeline.addLast("bookieProtoEncoder", new BookieProtoEncoding.RequestEncoder(extRegistry)); + pipeline.addLast("bookieProtoEncoder", new BookieProtoEncoding.RequestEncoder()); pipeline.addLast( "bookieProtoDecoder", - new BookieProtoEncoding.ResponseDecoder(extRegistry, useV2WireProtocol, shFactory != null)); + new BookieProtoEncoding.ResponseDecoder(useV2WireProtocol, shFactory != null)); pipeline.addLast("authHandler", new AuthHandler.ClientSideHandler(authProviderFactory, txnIdGenerator, connectionPeer, useV2WireProtocol)); pipeline.addLast("mainhandler", PerChannelBookieClient.this); @@ -710,26 +703,21 @@ void writeLac(final long ledgerId, final byte[] masterKey, final long lac, ByteB ctx, ledgerId, this)); // Build the request - BKPacketHeader.Builder headerBuilder = BKPacketHeader.newBuilder() + Request writeLacRequest = new Request(); + writeLacRequest.setHeader() .setVersion(ProtocolVersion.VERSION_THREE) .setOperation(OperationType.WRITE_LAC) .setTxnId(txnId); - ByteString body = ByteStringUtil.byteBufListToByteString(toSend); - toSend.retain(); - Runnable cleanupActionFailedBeforeWrite = toSend::release; - Runnable cleanupActionAfterWrite = cleanupActionFailedBeforeWrite; - WriteLacRequest.Builder writeLacBuilder = WriteLacRequest.newBuilder() + ByteBuf body = ByteBufList.coalesce(toSend); + Runnable releaseBody = body::release; + writeLacRequest.setWriteLacRequest() .setLedgerId(ledgerId) .setLac(lac) - .setMasterKey(UnsafeByteOperations.unsafeWrap(masterKey)) + .setMasterKey(masterKey) .setBody(body); + withRequestContext(writeLacRequest); - final Request writeLacRequest = withRequestContext(Request.newBuilder()) - .setHeader(headerBuilder) - .setWriteLacRequest(writeLacBuilder) - .build(); - writeAndFlush(channel, completionKey, writeLacRequest, false, cleanupActionFailedBeforeWrite, - cleanupActionAfterWrite); + writeAndFlush(channel, completionKey, writeLacRequest, false, releaseBody, releaseBody); } void forceLedger(final long ledgerId, ForceLedgerCallback cb, Object ctx) { @@ -749,17 +737,13 @@ void forceLedger(final long ledgerId, ForceLedgerCallback cb, Object ctx) { ctx, ledgerId, this)); // Build the request - BKPacketHeader.Builder headerBuilder = BKPacketHeader.newBuilder() + Request forceLedgerRequest = new Request(); + forceLedgerRequest.setHeader() .setVersion(ProtocolVersion.VERSION_THREE) .setOperation(OperationType.FORCE_LEDGER) .setTxnId(txnId); - ForceLedgerRequest.Builder writeLacBuilder = ForceLedgerRequest.newBuilder() - .setLedgerId(ledgerId); - - final Request forceLedgerRequest = withRequestContext(Request.newBuilder()) - .setHeader(headerBuilder) - .setForceLedgerRequest(writeLacBuilder) - .build(); + forceLedgerRequest.setForceLedgerRequest().setLedgerId(ledgerId); + withRequestContext(forceLedgerRequest); writeAndFlush(channel, completionKey, forceLedgerRequest); } @@ -813,38 +797,36 @@ void addEntry(final long ledgerId, byte[] masterKey, final long entryId, Referen completionKey = new TxnCompletionKey(txnId, OperationType.ADD_ENTRY); // Build the request and calculate the total size to be included in the packet. - BKPacketHeader.Builder headerBuilder = BKPacketHeader.newBuilder() + Request addEntryRequest = new Request(); + BKPacketHeader header = addEntryRequest.setHeader() .setVersion(ProtocolVersion.VERSION_THREE) .setOperation(OperationType.ADD_ENTRY) .setTxnId(txnId); if (((short) options & BookieProtocol.FLAG_HIGH_PRIORITY) == BookieProtocol.FLAG_HIGH_PRIORITY) { - headerBuilder.setPriority(DEFAULT_HIGH_PRIORITY_VALUE); + header.setPriority(DEFAULT_HIGH_PRIORITY_VALUE); } ByteBufList bufToSend = (ByteBufList) toSend; - ByteString body = ByteStringUtil.byteBufListToByteString(bufToSend); - bufToSend.retain(); - cleanupActionFailedBeforeWrite = bufToSend::release; + ByteBuf body = ByteBufList.coalesce(bufToSend); + cleanupActionFailedBeforeWrite = body::release; cleanupActionAfterWrite = cleanupActionFailedBeforeWrite; - AddRequest.Builder addBuilder = AddRequest.newBuilder() + AddRequest addRequestMsg = addEntryRequest.setAddRequest() .setLedgerId(ledgerId) .setEntryId(entryId) - .setMasterKey(UnsafeByteOperations.unsafeWrap(masterKey)) + .setMasterKey(masterKey) .setBody(body); if (((short) options & BookieProtocol.FLAG_RECOVERY_ADD) == BookieProtocol.FLAG_RECOVERY_ADD) { - addBuilder.setFlag(AddRequest.Flag.RECOVERY_ADD); + addRequestMsg.setFlag(AddRequest.Flag.RECOVERY_ADD); } if (!writeFlags.isEmpty()) { // add flags only if needed, in order to be able to talk with old bookies - addBuilder.setWriteFlags(WriteFlag.getWriteFlagsValue(writeFlags)); + addRequestMsg.setWriteFlags(WriteFlag.getWriteFlagsValue(writeFlags)); } - request = withRequestContext(Request.newBuilder()) - .setHeader(headerBuilder) - .setAddRequest(addBuilder) - .build(); + withRequestContext(addEntryRequest); + request = addEntryRequest; } putCompletionKeyValue(completionKey, @@ -867,16 +849,14 @@ public void readLac(final long ledgerId, ReadLacCallback cb, Object ctx) { completionKey = new TxnCompletionKey(txnId, OperationType.READ_LAC); // Build the request and calculate the total size to be included in the packet. - BKPacketHeader.Builder headerBuilder = BKPacketHeader.newBuilder() + Request readLacRequest = new Request(); + readLacRequest.setHeader() .setVersion(ProtocolVersion.VERSION_THREE) .setOperation(OperationType.READ_LAC) .setTxnId(txnId); - ReadLacRequest.Builder readLacBuilder = ReadLacRequest.newBuilder() - .setLedgerId(ledgerId); - request = withRequestContext(Request.newBuilder()) - .setHeader(headerBuilder) - .setReadLacRequest(readLacBuilder) - .build(); + readLacRequest.setReadLacRequest().setLedgerId(ledgerId); + withRequestContext(readLacRequest); + request = readLacRequest; } putCompletionKeyValue(completionKey, new ReadLacCompletion(completionKey, cb, @@ -891,14 +871,12 @@ public void getListOfEntriesOfLedger(final long ledgerId, GetListOfEntriesOfLedg completionKey, cb, ledgerId, this)); // Build the request. - BKPacketHeader.Builder headerBuilder = BKPacketHeader.newBuilder().setVersion(ProtocolVersion.VERSION_THREE) - .setOperation(OperationType.GET_LIST_OF_ENTRIES_OF_LEDGER).setTxnId(txnId); - - GetListOfEntriesOfLedgerRequest.Builder getListOfEntriesOfLedgerRequestBuilder = - GetListOfEntriesOfLedgerRequest.newBuilder().setLedgerId(ledgerId); - - final Request getListOfEntriesOfLedgerRequest = Request.newBuilder().setHeader(headerBuilder) - .setGetListOfEntriesOfLedgerRequest(getListOfEntriesOfLedgerRequestBuilder).build(); + Request getListOfEntriesOfLedgerRequest = new Request(); + getListOfEntriesOfLedgerRequest.setHeader() + .setVersion(ProtocolVersion.VERSION_THREE) + .setOperation(OperationType.GET_LIST_OF_ENTRIES_OF_LEDGER) + .setTxnId(txnId); + getListOfEntriesOfLedgerRequest.setGetListOfEntriesOfLedgerRequest().setLedgerId(ledgerId); writeAndFlush(channel, completionKey, getListOfEntriesOfLedgerRequest); } @@ -952,20 +930,21 @@ private void readEntryInternal(final long ledgerId, completionKey = new TxnCompletionKey(txnId, OperationType.READ_ENTRY); // Build the request and calculate the total size to be included in the packet. - BKPacketHeader.Builder headerBuilder = BKPacketHeader.newBuilder() + Request readEntryRequest = new Request(); + BKPacketHeader header = readEntryRequest.setHeader() .setVersion(ProtocolVersion.VERSION_THREE) .setOperation(OperationType.READ_ENTRY) .setTxnId(txnId); if (((short) flags & BookieProtocol.FLAG_HIGH_PRIORITY) == BookieProtocol.FLAG_HIGH_PRIORITY) { - headerBuilder.setPriority(DEFAULT_HIGH_PRIORITY_VALUE); + header.setPriority(DEFAULT_HIGH_PRIORITY_VALUE); } - ReadRequest.Builder readBuilder = ReadRequest.newBuilder() + ReadRequest readRequestMsg = readEntryRequest.setReadRequest() .setLedgerId(ledgerId) .setEntryId(entryId); if (null != previousLAC) { - readBuilder = readBuilder.setPreviousLAC(previousLAC); + readRequestMsg.setPreviousLAC(previousLAC); } if (null != timeOutInMillis) { @@ -975,7 +954,7 @@ private void readEntryInternal(final long ledgerId, ledgerId, entryId, null, ctx); return; } - readBuilder = readBuilder.setTimeOut(timeOutInMillis); + readRequestMsg.setTimeOut(timeOutInMillis); } if (piggyBackEntry) { @@ -985,24 +964,22 @@ private void readEntryInternal(final long ledgerId, ledgerId, entryId, null, ctx); return; } - readBuilder = readBuilder.setFlag(ReadRequest.Flag.ENTRY_PIGGYBACK); + readRequestMsg.setFlag(ReadRequest.Flag.ENTRY_PIGGYBACK); } // Only one flag can be set on the read requests if (((short) flags & BookieProtocol.FLAG_DO_FENCING) == BookieProtocol.FLAG_DO_FENCING) { - readBuilder.setFlag(ReadRequest.Flag.FENCE_LEDGER); + readRequestMsg.setFlag(ReadRequest.Flag.FENCE_LEDGER); if (masterKey == null) { cb.readEntryComplete(BKException.Code.IncorrectParameterException, ledgerId, entryId, null, ctx); return; } - readBuilder.setMasterKey(ByteString.copyFrom(masterKey)); + readRequestMsg.setMasterKey(masterKey); } - request = withRequestContext(Request.newBuilder()) - .setHeader(headerBuilder) - .setReadRequest(readBuilder) - .build(); + withRequestContext(readEntryRequest); + request = readEntryRequest; } ReadCompletion readCompletion = new ReadCompletion(completionKey, cb, ctx, ledgerId, entryId, this); @@ -1062,18 +1039,13 @@ public void getBookieInfo(final long requested, GetBookieInfoCallback cb, Object completionKey, cb, ctx, this)); // Build the request and calculate the total size to be included in the packet. - BKPacketHeader.Builder headerBuilder = BKPacketHeader.newBuilder() + Request getBookieInfoRequest = new Request(); + getBookieInfoRequest.setHeader() .setVersion(ProtocolVersion.VERSION_THREE) .setOperation(OperationType.GET_BOOKIE_INFO) .setTxnId(txnId); - - GetBookieInfoRequest.Builder getBookieInfoBuilder = GetBookieInfoRequest.newBuilder() - .setRequested(requested); - - final Request getBookieInfoRequest = withRequestContext(Request.newBuilder()) - .setHeader(headerBuilder) - .setGetBookieInfoRequest(getBookieInfoBuilder) - .build(); + getBookieInfoRequest.setGetBookieInfoRequest().setRequested(requested); + withRequestContext(getBookieInfoRequest); writeAndFlush(channel, completionKey, getBookieInfoRequest); } @@ -1697,26 +1669,22 @@ private long getTxnId() { return txnIdGenerator.incrementAndGet(); } - Request.Builder withRequestContext(Request.Builder builder) { + Request withRequestContext(Request request) { if (preserveMdcForTaskExecution) { - return appendRequestContext(builder); + return appendRequestContext(request); } - return builder; + return request; } - static Request.Builder appendRequestContext(Request.Builder builder) { + static Request appendRequestContext(Request request) { final Map mdcContextMap = MDC.getCopyOfContextMap(); if (mdcContextMap == null || mdcContextMap.isEmpty()) { - return builder; + return request; } for (Map.Entry kv : mdcContextMap.entrySet()) { - final BookkeeperProtocol.ContextPair context = BookkeeperProtocol.ContextPair.newBuilder() - .setKey(kv.getKey()) - .setValue(kv.getValue()) - .build(); - builder.addRequestContext(context); + request.addRequestContext().setKey(kv.getKey()).setValue(kv.getValue()); } - return builder; + return request; } ChannelFutureListener contextPreservingListener(ChannelFutureListener listener) { @@ -1878,15 +1846,15 @@ private void initiateTLS() { final CompletionKey completionKey = new TxnCompletionKey(txnId, OperationType.START_TLS); completionObjects.put(completionKey, new StartTLSCompletion(completionKey, this)); - BookkeeperProtocol.Request.Builder h = withRequestContext(BookkeeperProtocol.Request.newBuilder()); - BKPacketHeader.Builder headerBuilder = BKPacketHeader.newBuilder() + Request startTlsRequest = new Request(); + startTlsRequest.setHeader() .setVersion(ProtocolVersion.VERSION_THREE) .setOperation(OperationType.START_TLS) .setTxnId(txnId); - h.setHeader(headerBuilder.build()); - h.setStartTLSRequest(BookkeeperProtocol.StartTLSRequest.newBuilder().build()); + startTlsRequest.setStartTLSRequest(); + withRequestContext(startTlsRequest); state = ConnectionState.START_TLS; - writeAndFlush(channel, completionKey, h.build()); + writeAndFlush(channel, completionKey, startTlsRequest); } protected void failTLS(int rc) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadCompletion.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadCompletion.java index b34d4e370a6..bde804756ac 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadCompletion.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadCompletion.java @@ -67,7 +67,7 @@ public void setOutstanding() { @Override public void handleV2Response(long ledgerId, long entryId, - BookkeeperProtocol.StatusCode status, + StatusCode status, BookieProtocol.Response response) { perChannelBookieClient.readEntryOutstanding.dec(); if (!(response instanceof BookieProtocol.ReadResponse)) { @@ -79,14 +79,14 @@ public void handleV2Response(long ledgerId, long entryId, } @Override - public void handleV3Response(BookkeeperProtocol.Response response) { + public void handleV3Response(Response response) { perChannelBookieClient.readEntryOutstanding.dec(); - BookkeeperProtocol.ReadResponse readResponse = response.getReadResponse(); - BookkeeperProtocol.StatusCode status = response.getStatus() == BookkeeperProtocol.StatusCode.EOK + ReadResponse readResponse = response.getReadResponse(); + StatusCode status = response.getStatus() == StatusCode.EOK ? readResponse.getStatus() : response.getStatus(); ByteBuf buffer = Unpooled.EMPTY_BUFFER; if (readResponse.hasBody()) { - buffer = Unpooled.wrappedBuffer(readResponse.getBody().asReadOnlyByteBuffer()); + buffer = readResponse.getBodySlice(); } long maxLAC = INVALID_ENTRY_ID; if (readResponse.hasMaxLAC()) { @@ -105,7 +105,7 @@ public void handleV3Response(BookkeeperProtocol.Response response) { private void handleReadResponse(long ledgerId, long entryId, - BookkeeperProtocol.StatusCode status, + StatusCode status, ByteBuf buffer, long maxLAC, // max known lac piggy-back from bookies long lacUpdateTimestamp) { // the timestamp when the lac is updated. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java index 118d5a66da6..3e01ac6bafe 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java @@ -18,8 +18,8 @@ package org.apache.bookkeeper.proto; import com.google.common.base.Stopwatch; -import com.google.protobuf.ByteString; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufUtil; import io.netty.channel.Channel; import io.netty.util.ReferenceCountUtil; import java.io.IOException; @@ -31,11 +31,11 @@ import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.util.MathUtils; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ReadRequest; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ReadResponse; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Response; -import org.apache.bookkeeper.proto.BookkeeperProtocol.StatusCode; +import org.apache.bookkeeper.proto.ReadRequest; +import org.apache.bookkeeper.proto.ReadResponse; +import org.apache.bookkeeper.proto.Request; +import org.apache.bookkeeper.proto.Response; +import org.apache.bookkeeper.proto.StatusCode; import org.apache.bookkeeper.stats.OpStatsLogger; @CustomLog @@ -101,7 +101,7 @@ protected Long getPreviousLAC() { */ protected void handleReadResultForFenceRead( final ByteBuf entryBody, - final ReadResponse.Builder readResponseBuilder, + final ReadResponse readResponseBuilder, final long entryId, final Stopwatch startTimeSw) { // reset last phase start time to measure fence result waiting time @@ -150,7 +150,7 @@ public void onFailure(Throwable t) { * @return read response or null if it is a fence read operation. * @throws IOException */ - protected ReadResponse readEntry(ReadResponse.Builder readResponseBuilder, + protected ReadResponse readEntry(ReadResponse readResponseBuilder, long entryId, Stopwatch startTimeSw) throws IOException, BookieException { @@ -169,7 +169,7 @@ protected ReadResponse readEntry(ReadResponse.Builder readResponseBuilder, * @return read response or null if it is a fence read operation. * @throws IOException */ - protected ReadResponse readEntry(ReadResponse.Builder readResponseBuilder, + protected ReadResponse readEntry(ReadResponse readResponseBuilder, long entryId, boolean readLACPiggyBack, Stopwatch startTimeSw) @@ -180,7 +180,7 @@ protected ReadResponse readEntry(ReadResponse.Builder readResponseBuilder, return null; } else { try { - readResponseBuilder.setBody(ByteString.copyFrom(entryBody.nioBuffer())); + readResponseBuilder.setBody(ByteBufUtil.getBytes(entryBody)); if (readLACPiggyBack) { readResponseBuilder.setEntryId(entryId); } else { @@ -189,7 +189,7 @@ protected ReadResponse readEntry(ReadResponse.Builder readResponseBuilder, } registerSuccessfulEvent(readStats, startTimeSw); readResponseBuilder.setStatus(StatusCode.EOK); - return readResponseBuilder.build(); + return readResponseBuilder; } finally { ReferenceCountUtil.release(entryBody); } @@ -200,9 +200,9 @@ protected ReadResponse getReadResponse() { final Stopwatch startTimeSw = Stopwatch.createStarted(); final Channel channel = requestHandler.ctx().channel(); - final ReadResponse.Builder readResponse = ReadResponse.newBuilder() - .setLedgerId(ledgerId) - .setEntryId(entryId); + final ReadResponse readResponse = new ReadResponse() + .setLedgerId(ledgerId) + .setEntryId(entryId); try { // handle fence request if (RequestUtils.isFenceRequest(readRequest)) { @@ -217,7 +217,7 @@ protected ReadResponse getReadResponse() { .log("Fence ledger request received without master key"); throw BookieException.create(BookieException.Code.UnauthorizedAccessException); } else { - byte[] masterKey = readRequest.getMasterKey().toByteArray(); + byte[] masterKey = readRequest.getMasterKey(); fenceResult = requestProcessor.bookie.fenceLedger(ledgerId, masterKey); } } @@ -282,11 +282,10 @@ public void run() { } if (!isVersionCompatible()) { - ReadResponse readResponse = ReadResponse.newBuilder() - .setLedgerId(ledgerId) - .setEntryId(entryId) - .setStatus(StatusCode.EBADVERSION) - .build(); + ReadResponse readResponse = new ReadResponse() + .setLedgerId(ledgerId) + .setEntryId(entryId) + .setStatus(StatusCode.EBADVERSION); sendResponse(readResponse); return; } @@ -301,7 +300,7 @@ protected void executeOp() { } } - private void getFenceResponse(ReadResponse.Builder readResponse, + private void getFenceResponse(ReadResponse readResponse, ByteBuf entryBody, boolean fenceResult) { StatusCode status; @@ -310,7 +309,7 @@ private void getFenceResponse(ReadResponse.Builder readResponse, registerFailedEvent(requestProcessor.getRequestStats().getFenceReadWaitStats(), lastPhaseStartTime); } else { status = StatusCode.EOK; - readResponse.setBody(ByteString.copyFrom(entryBody.nioBuffer())); + readResponse.setBody(ByteBufUtil.getBytes(entryBody)); registerSuccessfulEvent(requestProcessor.getRequestStats().getFenceReadWaitStats(), lastPhaseStartTime); } @@ -321,7 +320,7 @@ private void getFenceResponse(ReadResponse.Builder readResponse, readResponse.setStatus(status); } - private void sendFenceResponse(ReadResponse.Builder readResponse, + private void sendFenceResponse(ReadResponse readResponse, ByteBuf entryBody, boolean fenceResult, Stopwatch startTimeSw) { @@ -330,26 +329,24 @@ private void sendFenceResponse(ReadResponse.Builder readResponse, // register fence read stat registerEvent(!fenceResult, requestProcessor.getRequestStats().getFenceReadEntryStats(), startTimeSw); // send the fence read response - sendResponse(readResponse.build()); + sendResponse(readResponse); } protected ReadResponse buildResponse( - ReadResponse.Builder readResponseBuilder, + ReadResponse readResponseBuilder, StatusCode statusCode, Stopwatch startTimeSw) { registerEvent(!statusCode.equals(StatusCode.EOK), readStats, startTimeSw); readResponseBuilder.setStatus(statusCode); - return readResponseBuilder.build(); + return readResponseBuilder; } protected void sendResponse(ReadResponse readResponse) { - Response.Builder response = Response.newBuilder() - .setHeader(getHeader()) - .setStatus(readResponse.getStatus()) - .setReadResponse(readResponse); - sendResponse(response.getStatus(), - response.build(), - reqStats); + Response response = new Response(); + response.setHeader().copyFrom(getHeader()); + response.setStatus(readResponse.getStatus()); + response.setReadResponse().copyFrom(readResponse); + sendResponse(response.getStatus(), response, reqStats); requestProcessor.onReadRequestFinish(); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacCompletion.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacCompletion.java index f86afdd499f..73c734c6f24 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacCompletion.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacCompletion.java @@ -61,19 +61,19 @@ public void errorOut(final int rc) { } @Override - public void handleV3Response(BookkeeperProtocol.Response response) { - BookkeeperProtocol.ReadLacResponse readLacResponse = response.getReadLacResponse(); + public void handleV3Response(Response response) { + ReadLacResponse readLacResponse = response.getReadLacResponse(); ByteBuf lacBuffer = Unpooled.EMPTY_BUFFER; ByteBuf lastEntryBuffer = Unpooled.EMPTY_BUFFER; - BookkeeperProtocol.StatusCode status = response.getStatus() == BookkeeperProtocol.StatusCode.EOK + StatusCode status = response.getStatus() == StatusCode.EOK ? readLacResponse.getStatus() : response.getStatus(); if (readLacResponse.hasLacBody()) { - lacBuffer = Unpooled.wrappedBuffer(readLacResponse.getLacBody().asReadOnlyByteBuffer()); + lacBuffer = readLacResponse.getLacBodySlice(); } if (readLacResponse.hasLastEntryBody()) { - lastEntryBuffer = Unpooled.wrappedBuffer(readLacResponse.getLastEntryBody().asReadOnlyByteBuffer()); + lastEntryBuffer = readLacResponse.getLastEntryBodySlice(); } logEvent(status).log("Got response from bookie"); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacProcessorV3.java index 0fb27e938e5..04e70d329e8 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacProcessorV3.java @@ -20,8 +20,8 @@ */ package org.apache.bookkeeper.proto; -import com.google.protobuf.ByteString; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufUtil; import io.netty.util.ReferenceCountUtil; import java.io.IOException; import java.util.concurrent.TimeUnit; @@ -29,11 +29,11 @@ import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.common.util.MathUtils; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ReadLacRequest; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ReadLacResponse; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Response; -import org.apache.bookkeeper.proto.BookkeeperProtocol.StatusCode; +import org.apache.bookkeeper.proto.ReadLacRequest; +import org.apache.bookkeeper.proto.ReadLacResponse; +import org.apache.bookkeeper.proto.Request; +import org.apache.bookkeeper.proto.Response; +import org.apache.bookkeeper.proto.StatusCode; /** * A read processor for v3 last add confirmed messages. @@ -52,11 +52,11 @@ private ReadLacResponse getReadLacResponse() { ReadLacRequest readLacRequest = request.getReadLacRequest(); long ledgerId = readLacRequest.getLedgerId(); - final ReadLacResponse.Builder readLacResponse = ReadLacResponse.newBuilder().setLedgerId(ledgerId); + final ReadLacResponse readLacResponse = new ReadLacResponse().setLedgerId(ledgerId); if (!isVersionCompatible()) { readLacResponse.setStatus(StatusCode.EBADVERSION); - return readLacResponse.build(); + return readLacResponse; } log.debug().attr("request", request).log("Received ReadLac request"); @@ -66,7 +66,7 @@ private ReadLacResponse getReadLacResponse() { try { lac = requestProcessor.bookie.getExplicitLac(ledgerId); if (lac != null) { - readLacResponse.setLacBody(ByteString.copyFrom(lac.nioBuffer())); + readLacResponse.setLacBody(ByteBufUtil.getBytes(lac)); } } catch (Bookie.NoLedgerException e) { status = StatusCode.ENOLEDGER; @@ -93,7 +93,7 @@ private ReadLacResponse getReadLacResponse() { try { lastEntry = requestProcessor.bookie.readEntry(ledgerId, BookieProtocol.LAST_ADD_CONFIRMED); if (lastEntry != null) { - readLacResponse.setLastEntryBody(ByteString.copyFrom(lastEntry.nioBuffer())); + readLacResponse.setLastEntryBody(ByteBufUtil.getBytes(lastEntry)); } } catch (Bookie.NoLedgerException e) { status = StatusCode.ENOLEDGER; @@ -130,7 +130,7 @@ private ReadLacResponse getReadLacResponse() { } // Finally set the status and return readLacResponse.setStatus(status); - return readLacResponse.build(); + return readLacResponse; } @Override @@ -140,12 +140,12 @@ public void run() { } private void sendResponse(ReadLacResponse readLacResponse) { - Response.Builder response = Response.newBuilder() - .setHeader(getHeader()) - .setStatus(readLacResponse.getStatus()) - .setReadLacResponse(readLacResponse); + Response response = new Response(); + response.setHeader().copyFrom(getHeader()); + response.setStatus(readLacResponse.getStatus()); + response.setReadLacResponse().copyFrom(readLacResponse); sendResponse(response.getStatus(), - response.build(), + response, requestProcessor.getRequestStats().getReadLacRequestStats()); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/RequestUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/RequestUtils.java index 9b8533da0c6..210ff41a71a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/RequestUtils.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/RequestUtils.java @@ -24,27 +24,27 @@ */ class RequestUtils { - public static boolean isFenceRequest(BookkeeperProtocol.ReadRequest readRequest) { - return hasFlag(readRequest, BookkeeperProtocol.ReadRequest.Flag.FENCE_LEDGER); + public static boolean isFenceRequest(ReadRequest readRequest) { + return hasFlag(readRequest, ReadRequest.Flag.FENCE_LEDGER); } - public static boolean isLongPollReadRequest(BookkeeperProtocol.ReadRequest readRequest) { + public static boolean isLongPollReadRequest(ReadRequest readRequest) { return !isFenceRequest(readRequest) && readRequest.hasPreviousLAC(); } - public static boolean isHighPriority(BookkeeperProtocol.Request request) { + public static boolean isHighPriority(Request request) { return request.getHeader().getPriority() > 0; } - public static boolean shouldPiggybackEntry(BookkeeperProtocol.ReadRequest readRequest) { - return hasFlag(readRequest, BookkeeperProtocol.ReadRequest.Flag.ENTRY_PIGGYBACK); + public static boolean shouldPiggybackEntry(ReadRequest readRequest) { + return hasFlag(readRequest, ReadRequest.Flag.ENTRY_PIGGYBACK); } - static boolean hasFlag(BookkeeperProtocol.ReadRequest request, BookkeeperProtocol.ReadRequest.Flag flag) { + static boolean hasFlag(ReadRequest request, ReadRequest.Flag flag) { return request.hasFlag() && request.getFlag() == flag; } - static boolean hasFlag(BookkeeperProtocol.AddRequest request, BookkeeperProtocol.AddRequest.Flag flag) { + static boolean hasFlag(AddRequest request, AddRequest.Flag flag) { return request.hasFlag() && request.getFlag() == flag; } @@ -53,11 +53,11 @@ static boolean hasFlag(BookkeeperProtocol.AddRequest request, BookkeeperProtocol * masterKey contains the password of the ledger and body is customer data, * so it is not appropriate to have these in logs or system output. */ - public static String toSafeString(BookkeeperProtocol.Request request) { + public static String toSafeString(Request request) { MoreObjects.ToStringHelper stringHelper = MoreObjects.toStringHelper(request); - BookkeeperProtocol.BKPacketHeader header = request.getHeader(); + BKPacketHeader header = request.getHeader(); if (request.hasAddRequest()) { - BookkeeperProtocol.AddRequest addRequest = request.getAddRequest(); + AddRequest addRequest = request.getAddRequest(); includeHeaderFields(stringHelper, header); stringHelper.add("ledgerId", addRequest.getLedgerId()); stringHelper.add("entryId", addRequest.getEntryId()); @@ -69,7 +69,7 @@ public static String toSafeString(BookkeeperProtocol.Request request) { } return stringHelper.toString(); } else if (request.hasReadRequest()) { - BookkeeperProtocol.ReadRequest readRequest = request.getReadRequest(); + ReadRequest readRequest = request.getReadRequest(); includeHeaderFields(stringHelper, header); stringHelper.add("ledgerId", readRequest.getLedgerId()); stringHelper.add("entryId", readRequest.getEntryId()); @@ -84,13 +84,13 @@ public static String toSafeString(BookkeeperProtocol.Request request) { } return stringHelper.toString(); } else if (request.hasWriteLacRequest()) { - BookkeeperProtocol.WriteLacRequest writeLacRequest = request.getWriteLacRequest(); + WriteLacRequest writeLacRequest = request.getWriteLacRequest(); includeHeaderFields(stringHelper, header); stringHelper.add("ledgerId", writeLacRequest.getLedgerId()); stringHelper.add("lac", writeLacRequest.getLac()); return stringHelper.toString(); } else if (request.hasForceLedgerRequest()) { - BookkeeperProtocol.ForceLedgerRequest forceLedgerRequest = request.getForceLedgerRequest(); + ForceLedgerRequest forceLedgerRequest = request.getForceLedgerRequest(); includeHeaderFields(stringHelper, header); stringHelper.add("ledgerId", forceLedgerRequest.getLedgerId()); return stringHelper.toString(); @@ -100,7 +100,7 @@ public static String toSafeString(BookkeeperProtocol.Request request) { } private static void includeHeaderFields(MoreObjects.ToStringHelper stringHelper, - BookkeeperProtocol.BKPacketHeader header) { + BKPacketHeader header) { stringHelper.add("version", header.getVersion()); stringHelper.add("operation", header.getOperation()); stringHelper.add("txnId", header.getTxnId()); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/StartTLSCompletion.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/StartTLSCompletion.java index a90a1c90351..e537f2082e6 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/StartTLSCompletion.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/StartTLSCompletion.java @@ -52,8 +52,8 @@ public void errorOut(final int rc) { } @Override - public void handleV3Response(BookkeeperProtocol.Response response) { - BookkeeperProtocol.StatusCode status = response.getStatus(); + public void handleV3Response(Response response) { + StatusCode status = response.getStatus(); logEvent(status).log("Got response from bookie"); @@ -65,7 +65,7 @@ public void handleV3Response(BookkeeperProtocol.Response response) { if (perChannelBookieClient.state != PerChannelBookieClient.ConnectionState.START_TLS) { log.error("Connection state changed before TLS response received"); perChannelBookieClient.failTLS(BKException.Code.BookieHandleNotAvailableException); - } else if (status != BookkeeperProtocol.StatusCode.EOK) { + } else if (status != StatusCode.EOK) { log.error().attr("status", status).log("Client received error during TLS negotiation"); perChannelBookieClient.failTLS(BKException.Code.SecurityException); } else { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/TxnCompletionKey.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/TxnCompletionKey.java index bbf64681e1d..b5c6cc6f23a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/TxnCompletionKey.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/TxnCompletionKey.java @@ -24,7 +24,7 @@ class TxnCompletionKey extends CompletionKey { final long txnId; - public TxnCompletionKey(long txnId, BookkeeperProtocol.OperationType operationType) { + public TxnCompletionKey(long txnId, OperationType operationType) { super(operationType); this.txnId = txnId; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3.java index 6bebfcc4b9e..8ccc76e4c24 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3.java @@ -21,7 +21,6 @@ package org.apache.bookkeeper.proto; import io.netty.buffer.ByteBuf; -import io.netty.buffer.Unpooled; import java.io.IOException; import java.util.EnumSet; import java.util.concurrent.TimeUnit; @@ -31,11 +30,11 @@ import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.common.util.MathUtils; import org.apache.bookkeeper.net.BookieId; -import org.apache.bookkeeper.proto.BookkeeperProtocol.AddRequest; -import org.apache.bookkeeper.proto.BookkeeperProtocol.AddResponse; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Response; -import org.apache.bookkeeper.proto.BookkeeperProtocol.StatusCode; +import org.apache.bookkeeper.proto.AddRequest; +import org.apache.bookkeeper.proto.AddResponse; +import org.apache.bookkeeper.proto.Request; +import org.apache.bookkeeper.proto.Response; +import org.apache.bookkeeper.proto.StatusCode; import org.apache.bookkeeper.stats.OpStatsLogger; @CustomLog @@ -54,13 +53,13 @@ private AddResponse getAddResponse() { long ledgerId = addRequest.getLedgerId(); long entryId = addRequest.getEntryId(); - final AddResponse.Builder addResponse = AddResponse.newBuilder() + final AddResponse addResponse = new AddResponse() .setLedgerId(ledgerId) .setEntryId(entryId); if (!isVersionCompatible()) { addResponse.setStatus(StatusCode.EBADVERSION); - return addResponse.build(); + return addResponse; } if (requestProcessor.getBookie().isReadOnly() @@ -68,7 +67,7 @@ private AddResponse getAddResponse() { && requestProcessor.getBookie().isAvailableForHighPriorityWrites())) { log.warn("BookieServer is running as readonly mode, so rejecting the request from the client!"); addResponse.setStatus(StatusCode.EREADONLY); - return addResponse.build(); + return addResponse; } BookkeeperInternalCallbacks.WriteCallback wcb = new BookkeeperInternalCallbacks.WriteCallback() { @@ -96,11 +95,10 @@ public void writeComplete(int rc, long ledgerId, long entryId, break; } addResponse.setStatus(status); - Response.Builder response = Response.newBuilder() - .setHeader(getHeader()) - .setStatus(addResponse.getStatus()) - .setAddResponse(addResponse); - Response resp = response.build(); + Response resp = new Response(); + resp.setHeader().copyFrom(getHeader()); + resp.setStatus(addResponse.getStatus()); + resp.setAddResponse().copyFrom(addResponse); sendResponse(status, resp, requestProcessor.getRequestStats().getAddRequestStats()); } }; @@ -112,8 +110,8 @@ public void writeComplete(int rc, long ledgerId, long entryId, } final boolean ackBeforeSync = writeFlags.contains(WriteFlag.DEFERRED_SYNC); StatusCode status = null; - byte[] masterKey = addRequest.getMasterKey().toByteArray(); - ByteBuf entryToAdd = Unpooled.wrappedBuffer(addRequest.getBody().asReadOnlyByteBuffer()); + byte[] masterKey = addRequest.getMasterKey(); + ByteBuf entryToAdd = addRequest.getBodySlice(); try { if (RequestUtils.hasFlag(addRequest, AddRequest.Flag.RECOVERY_ADD)) { requestProcessor.getBookie().recoveryAddEntry(entryToAdd, wcb, @@ -167,7 +165,7 @@ public void writeComplete(int rc, long ledgerId, long entryId, // doesn't return a response back to the caller. if (!status.equals(StatusCode.EOK)) { addResponse.setStatus(status); - return addResponse.build(); + return addResponse; } return null; } @@ -179,11 +177,10 @@ public void run() { AddResponse addResponse = getAddResponse(); if (null != addResponse) { // This means there was an error and we should send this back. - Response.Builder response = Response.newBuilder() - .setHeader(getHeader()) - .setStatus(addResponse.getStatus()) - .setAddResponse(addResponse); - Response resp = response.build(); + Response resp = new Response(); + resp.setHeader().copyFrom(getHeader()); + resp.setStatus(addResponse.getStatus()); + resp.setAddResponse().copyFrom(addResponse); sendResponse(addResponse.getStatus(), resp, requestProcessor.getRequestStats().getAddRequestStats()); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacCompletion.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacCompletion.java index c5339dcd3e1..599ab36c6e4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacCompletion.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacCompletion.java @@ -61,9 +61,9 @@ public void errorOut(final int rc) { } @Override - public void handleV3Response(BookkeeperProtocol.Response response) { - BookkeeperProtocol.WriteLacResponse writeLacResponse = response.getWriteLacResponse(); - BookkeeperProtocol.StatusCode status = response.getStatus() == BookkeeperProtocol.StatusCode.EOK + public void handleV3Response(Response response) { + WriteLacResponse writeLacResponse = response.getWriteLacResponse(); + StatusCode status = response.getStatus() == StatusCode.EOK ? writeLacResponse.getStatus() : response.getStatus(); long ledgerId = writeLacResponse.getLedgerId(); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java index b22fa5b8626..9fcfa1f44dd 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java @@ -20,19 +20,18 @@ */ package org.apache.bookkeeper.proto; -import io.netty.buffer.Unpooled; +import io.netty.buffer.ByteBuf; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.concurrent.TimeUnit; import lombok.CustomLog; import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.common.util.MathUtils; import org.apache.bookkeeper.net.BookieId; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Response; -import org.apache.bookkeeper.proto.BookkeeperProtocol.StatusCode; -import org.apache.bookkeeper.proto.BookkeeperProtocol.WriteLacRequest; -import org.apache.bookkeeper.proto.BookkeeperProtocol.WriteLacResponse; +import org.apache.bookkeeper.proto.Request; +import org.apache.bookkeeper.proto.Response; +import org.apache.bookkeeper.proto.StatusCode; +import org.apache.bookkeeper.proto.WriteLacRequest; +import org.apache.bookkeeper.proto.WriteLacResponse; @CustomLog @@ -50,17 +49,17 @@ private WriteLacResponse getWriteLacResponse() { long lac = writeLacRequest.getLac(); long ledgerId = writeLacRequest.getLedgerId(); - final WriteLacResponse.Builder writeLacResponse = WriteLacResponse.newBuilder().setLedgerId(ledgerId); + final WriteLacResponse writeLacResponse = new WriteLacResponse().setLedgerId(ledgerId); if (!isVersionCompatible()) { writeLacResponse.setStatus(StatusCode.EBADVERSION); - return writeLacResponse.build(); + return writeLacResponse; } if (requestProcessor.bookie.isReadOnly()) { log.warn("BookieServer is running as readonly mode, so rejecting the request from the client!"); writeLacResponse.setStatus(StatusCode.EREADONLY); - return writeLacResponse.build(); + return writeLacResponse; } BookkeeperInternalCallbacks.WriteCallback writeCallback = new BookkeeperInternalCallbacks.WriteCallback() { @@ -87,21 +86,20 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Ob break; } writeLacResponse.setStatus(status); - Response.Builder response = Response.newBuilder() - .setHeader(getHeader()) - .setStatus(writeLacResponse.getStatus()) - .setWriteLacResponse(writeLacResponse); - Response resp = response.build(); + Response resp = new Response(); + resp.setHeader().copyFrom(getHeader()); + resp.setStatus(writeLacResponse.getStatus()); + resp.setWriteLacResponse().copyFrom(writeLacResponse); sendResponse(status, resp, requestProcessor.getRequestStats().getWriteLacRequestStats()); } }; StatusCode status = null; - ByteBuffer lacToAdd = writeLacRequest.getBody().asReadOnlyByteBuffer(); - byte[] masterKey = writeLacRequest.getMasterKey().toByteArray(); + ByteBuf lacToAdd = writeLacRequest.getBodySlice(); + byte[] masterKey = writeLacRequest.getMasterKey(); try { - requestProcessor.bookie.setExplicitLac(Unpooled.wrappedBuffer(lacToAdd), + requestProcessor.bookie.setExplicitLac(lacToAdd, writeCallback, requestHandler, masterKey); status = StatusCode.EOK; } catch (BookieException.LedgerFencedAndDeletedException e) { @@ -149,7 +147,7 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Ob requestProcessor.getRequestStats().getWriteLacStats() .registerFailedEvent(MathUtils.elapsedNanos(startTimeNanos), TimeUnit.NANOSECONDS); writeLacResponse.setStatus(status); - return writeLacResponse.build(); + return writeLacResponse; } return null; } @@ -158,11 +156,10 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Ob public void run() { WriteLacResponse writeLacResponse = getWriteLacResponse(); if (null != writeLacResponse) { - Response.Builder response = Response.newBuilder() - .setHeader(getHeader()) - .setStatus(writeLacResponse.getStatus()) - .setWriteLacResponse(writeLacResponse); - Response resp = response.build(); + Response resp = new Response(); + resp.setHeader().copyFrom(getHeader()); + resp.setStatus(writeLacResponse.getStatus()); + resp.setWriteLacResponse().copyFrom(writeLacResponse); sendResponse( writeLacResponse.getStatus(), resp, diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/StringEntryFormatter.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/StringEntryFormatter.java index 14ef44b7812..a931cfed2b7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/StringEntryFormatter.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/StringEntryFormatter.java @@ -21,8 +21,8 @@ package org.apache.bookkeeper.util; -import com.google.protobuf.ByteString; import java.io.IOException; +import java.nio.charset.StandardCharsets; /** * A String-based entry formatter. @@ -30,7 +30,7 @@ public class StringEntryFormatter extends EntryFormatter { @Override public void formatEntry(byte[] data) { - System.out.println(ByteString.copyFrom(data).toStringUtf8()); + System.out.println(new String(data, StandardCharsets.UTF_8)); } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/StringUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/StringUtils.java index edbe49989d2..8922cd0d632 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/StringUtils.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/StringUtils.java @@ -19,7 +19,8 @@ */ import java.io.IOException; -import org.apache.bookkeeper.proto.BookkeeperProtocol; +import org.apache.bookkeeper.proto.BKPacketHeader; +import org.apache.bookkeeper.proto.Request; /** * Provided utilities for parsing network addresses, ledger-id from node paths @@ -169,8 +170,8 @@ public static long stringToHierarchicalLedgerId(String...levelNodes) throws IOEx * @return string representation of request */ public static String requestToString(Object request) { - if (request instanceof BookkeeperProtocol.Request) { - BookkeeperProtocol.BKPacketHeader header = ((BookkeeperProtocol.Request) request).getHeader(); + if (request instanceof Request) { + BKPacketHeader header = ((Request) request).getHeader(); return String.format("Req(txnId=%d,op=%s,version=%s)", header.getTxnId(), header.getOperation(), header.getVersion()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java index c402451588b..f9c7babc643 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestGetBookieInfoTimeout.java @@ -41,7 +41,7 @@ import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookieClientImpl; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GetBookieInfoCallback; -import org.apache.bookkeeper.proto.BookkeeperProtocol; +import org.apache.bookkeeper.proto.GetBookieInfoRequest; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.After; @@ -110,8 +110,8 @@ public void testGetBookieInfoTimeout() throws Exception { // try to get bookie info from the sleeping bookie. It should fail with timeout error BookieClient bc = new BookieClientImpl(cConf, eventLoopGroup, UnpooledByteBufAllocator.DEFAULT, executor, scheduler, NullStatsLogger.INSTANCE, bkc.getBookieAddressResolver()); - long flags = BookkeeperProtocol.GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE - | BookkeeperProtocol.GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE; + long flags = GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE + | GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE; class CallbackObj { int rc; @@ -133,10 +133,10 @@ public void getBookieInfoComplete(int rc, BookieInfo bInfo, Object ctx) { CallbackObj obj = (CallbackObj) ctx; obj.rc = rc; if (rc == Code.OK) { - if ((obj.requested & BookkeeperProtocol.GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE) != 0) { + if ((obj.requested & GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE) != 0) { obj.freeDiskSpace = bInfo.getFreeDiskSpace(); } - if ((obj.requested & BookkeeperProtocol.GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE) + if ((obj.requested & GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE) != 0) { obj.totalDiskCapacity = bInfo.getTotalDiskSpace(); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplicationWithMock.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplicationWithMock.java index c618aa4a15b..9f1d7465e9d 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplicationWithMock.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplicationWithMock.java @@ -59,7 +59,7 @@ public void testRecoverLedgerFragmentEntrySendRightRequestWithFlag() throws Exce doAnswer(invocationOnMock -> { ByteBuf toSend = invocationOnMock.getArgument(4); BookieProtoEncoding.RequestEnDeCoderPreV3 deCoderPreV3 = - new BookieProtoEncoding.RequestEnDeCoderPreV3(null); + new BookieProtoEncoding.RequestEnDeCoderPreV3(); toSend.readerIndex(4); BookieProtocol.ParsedAddRequest request = (BookieProtocol.ParsedAddRequest) deCoderPreV3.decode(toSend); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieProtoEncodingTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieProtoEncodingTest.java index ba75e4b75a6..d628d279aa5 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieProtoEncodingTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieProtoEncodingTest.java @@ -27,9 +27,6 @@ import static org.mockito.Mockito.when; import com.google.common.collect.Lists; -import com.google.protobuf.ByteString; -import com.google.protobuf.ExtensionRegistry; -import com.google.protobuf.InvalidProtocolBufferException; import io.netty.buffer.ByteBuf; import io.netty.buffer.UnpooledByteBufAllocator; import io.netty.channel.ChannelHandlerContext; @@ -40,13 +37,8 @@ import org.apache.bookkeeper.proto.BookieProtoEncoding.ResponseEnDeCoderPreV3; import org.apache.bookkeeper.proto.BookieProtoEncoding.ResponseEnDecoderV3; import org.apache.bookkeeper.proto.BookieProtocol.AddResponse; -import org.apache.bookkeeper.proto.BookkeeperProtocol.AddRequest.Flag; -import org.apache.bookkeeper.proto.BookkeeperProtocol.BKPacketHeader; -import org.apache.bookkeeper.proto.BookkeeperProtocol.OperationType; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ProtocolVersion; -import org.apache.bookkeeper.proto.BookkeeperProtocol.StatusCode; +import org.apache.bookkeeper.proto.AddRequest.Flag; import org.apache.bookkeeper.util.ByteBufList; -import org.junit.Before; import org.junit.Test; /** @@ -54,13 +46,6 @@ */ public class BookieProtoEncodingTest { - private ExtensionRegistry registry; - - @Before - public void setup() { - this.registry = ExtensionRegistry.newInstance(); - } - @Test public void testV3ResponseDecoderNoFallback() throws Exception { AddResponse v2Resp = AddResponse.create( @@ -69,19 +54,16 @@ public void testV3ResponseDecoderNoFallback() throws Exception { 1L, 2L); - BookkeeperProtocol.Response v3Resp = BookkeeperProtocol.Response.newBuilder() - .setHeader(BKPacketHeader.newBuilder() + Response v3Resp = new Response(); + v3Resp.setHeader() .setVersion(ProtocolVersion.VERSION_THREE) .setTxnId(1L) - .setOperation(OperationType.ADD_ENTRY) - .build()) - .setStatus(StatusCode.EOK) - .setAddResponse(BookkeeperProtocol.AddResponse.newBuilder() + .setOperation(OperationType.ADD_ENTRY); + v3Resp.setStatus(StatusCode.EOK); + v3Resp.setAddResponse() .setStatus(StatusCode.EOK) .setLedgerId(1L) - .setEntryId(2L) - .build()) - .build(); + .setEntryId(2L); List outList = Lists.newArrayList(); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); @@ -90,16 +72,16 @@ public void testV3ResponseDecoderNoFallback() throws Exception { return null; }); - ResponseEnDeCoderPreV3 v2Encoder = new ResponseEnDeCoderPreV3(registry); - ResponseEnDecoderV3 v3Encoder = new ResponseEnDecoderV3(registry); + ResponseEnDeCoderPreV3 v2Encoder = new ResponseEnDeCoderPreV3(); + ResponseEnDecoderV3 v3Encoder = new ResponseEnDecoderV3(); - ResponseDecoder v3Decoder = new ResponseDecoder(registry, false, false); + ResponseDecoder v3Decoder = new ResponseDecoder(false, false); try { v3Decoder.channelRead(ctx, v2Encoder.encode(v2Resp, UnpooledByteBufAllocator.DEFAULT) ); fail("V3 response decoder should fail on decoding v2 response"); - } catch (InvalidProtocolBufferException e) { + } catch (RuntimeException e) { // expected } assertEquals(0, outList.size()); @@ -112,22 +94,20 @@ public void testV3ResponseDecoderNoFallback() throws Exception { @Test(expected = IllegalStateException.class) public void testV2RequestDecoderThrowExceptionOnUnknownRequests() throws Exception { - RequestEnDeCoderPreV3 v2ReqEncoder = new RequestEnDeCoderPreV3(registry); - RequestEnDecoderV3 v3ReqEncoder = new RequestEnDecoderV3(registry); + RequestEnDeCoderPreV3 v2ReqEncoder = new RequestEnDeCoderPreV3(); + RequestEnDecoderV3 v3ReqEncoder = new RequestEnDecoderV3(); - BookkeeperProtocol.Request v3Req = BookkeeperProtocol.Request.newBuilder() - .setHeader(BKPacketHeader.newBuilder() + Request v3Req = new Request(); + v3Req.setHeader() .setVersion(ProtocolVersion.VERSION_THREE) .setTxnId(1L) - .setOperation(OperationType.ADD_ENTRY) - .build()) - .setAddRequest(BookkeeperProtocol.AddRequest.newBuilder() + .setOperation(OperationType.ADD_ENTRY); + v3Req.setAddRequest() .setLedgerId(1L) .setEntryId(2L) - .setMasterKey(ByteString.copyFrom("", UTF_8)) + .setMasterKey("".getBytes(UTF_8)) .setFlag(Flag.RECOVERY_ADD) - .setBody(ByteString.copyFrom("test", UTF_8))) - .build(); + .setBody("test".getBytes(UTF_8)); v2ReqEncoder.decode((ByteBuf) v3ReqEncoder.encode(v3Req, UnpooledByteBufAllocator.DEFAULT)); @@ -135,7 +115,7 @@ public void testV2RequestDecoderThrowExceptionOnUnknownRequests() throws Excepti @Test public void testV2BatchReadRequest() throws Exception { - RequestEnDeCoderPreV3 v2ReqEncoder = new RequestEnDeCoderPreV3(registry); + RequestEnDeCoderPreV3 v2ReqEncoder = new RequestEnDeCoderPreV3(); BookieProtocol.BatchedReadRequest req = BookieProtocol.BatchedReadRequest.create( BookieProtocol.CURRENT_PROTOCOL_VERSION, 1L, 1L, FLAG_NONE, null, 1L, 10, 1024L); // Capture expected values before encode() recycles the request. @@ -155,7 +135,7 @@ public void testV2BatchReadRequest() throws Exception { @Test public void testV2BatchReadResponse() throws Exception { - ResponseEnDeCoderPreV3 v2ReqEncoder = new ResponseEnDeCoderPreV3(registry); + ResponseEnDeCoderPreV3 v2ReqEncoder = new ResponseEnDeCoderPreV3(); ByteBuf first = UnpooledByteBufAllocator.DEFAULT.buffer(4).writeInt(10); ByteBuf second = UnpooledByteBufAllocator.DEFAULT.buffer(8).writeLong(10L); ByteBufList data = ByteBufList.get(first, second); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3Test.java index 3bc9cbee427..fd873e77cf6 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3Test.java @@ -38,13 +38,13 @@ import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieImpl; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; -import org.apache.bookkeeper.proto.BookkeeperProtocol.BKPacketHeader; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ForceLedgerRequest; -import org.apache.bookkeeper.proto.BookkeeperProtocol.OperationType; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ProtocolVersion; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Response; -import org.apache.bookkeeper.proto.BookkeeperProtocol.StatusCode; +import org.apache.bookkeeper.proto.BKPacketHeader; +import org.apache.bookkeeper.proto.ForceLedgerRequest; +import org.apache.bookkeeper.proto.OperationType; +import org.apache.bookkeeper.proto.ProtocolVersion; +import org.apache.bookkeeper.proto.Request; +import org.apache.bookkeeper.proto.Response; +import org.apache.bookkeeper.proto.StatusCode; import org.apache.bookkeeper.stats.NullStatsLogger; import org.junit.Before; import org.junit.Test; @@ -64,16 +64,12 @@ public class ForceLedgerProcessorV3Test { @Before public void setup() { - request = Request.newBuilder() - .setHeader(BKPacketHeader.newBuilder() + request = new Request(); + request.setHeader() .setTxnId(System.currentTimeMillis()) .setVersion(ProtocolVersion.VERSION_THREE) - .setOperation(OperationType.ADD_ENTRY) - .build()) - .setForceLedgerRequest(ForceLedgerRequest.newBuilder() - .setLedgerId(System.currentTimeMillis()) - .build()) - .build(); + .setOperation(OperationType.ADD_ENTRY); + request.setForceLedgerRequest().setLedgerId(System.currentTimeMillis()); channel = mock(Channel.class); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/GetBookieInfoProcessorV3Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/GetBookieInfoProcessorV3Test.java index 5e986515e92..d7083110a7f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/GetBookieInfoProcessorV3Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/GetBookieInfoProcessorV3Test.java @@ -79,20 +79,13 @@ public void setup() { @Test public void testGetBookieInfoProcessorStats() throws IOException { - final BookkeeperProtocol.BKPacketHeader.Builder headerBuilder = - BookkeeperProtocol.BKPacketHeader.newBuilder() - .setVersion(BookkeeperProtocol.ProtocolVersion.VERSION_THREE) - .setOperation(BookkeeperProtocol.OperationType.GET_BOOKIE_INFO) + final Request getBookieInfoRequest = new Request(); + getBookieInfoRequest.setHeader() + .setVersion(ProtocolVersion.VERSION_THREE) + .setOperation(OperationType.GET_BOOKIE_INFO) .setTxnId(0); - - final BookkeeperProtocol.GetBookieInfoRequest.Builder getBookieInfoBuilder = - BookkeeperProtocol.GetBookieInfoRequest.newBuilder() - .setRequested(BookkeeperProtocol.GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE); - - final BookkeeperProtocol.Request getBookieInfoRequest = BookkeeperProtocol.Request.newBuilder() - .setHeader(headerBuilder) - .setGetBookieInfoRequest(getBookieInfoBuilder) - .build(); + getBookieInfoRequest.setGetBookieInfoRequest() + .setRequested(GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE); GetBookieInfoProcessorV3 getBookieInfo = new GetBookieInfoProcessorV3( getBookieInfoRequest, requestHandler, requestProcessor); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3Test.java index 33a4fdc8295..fd6ddda0d26 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3Test.java @@ -24,7 +24,6 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import com.google.protobuf.ByteString; import io.netty.buffer.Unpooled; import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; @@ -34,11 +33,11 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.bookie.Bookie; -import org.apache.bookkeeper.proto.BookkeeperProtocol.BKPacketHeader; -import org.apache.bookkeeper.proto.BookkeeperProtocol.OperationType; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ProtocolVersion; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ReadRequest; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; +import org.apache.bookkeeper.proto.BKPacketHeader; +import org.apache.bookkeeper.proto.OperationType; +import org.apache.bookkeeper.proto.ProtocolVersion; +import org.apache.bookkeeper.proto.ReadRequest; +import org.apache.bookkeeper.proto.Request; import org.apache.bookkeeper.stats.NullStatsLogger; import org.junit.After; import org.junit.Before; @@ -68,20 +67,17 @@ public void teardown() { @Test public void testWatchIsCancelledOnTimeout() throws Exception { - Request request = Request.newBuilder() - .setHeader(BKPacketHeader.newBuilder() - .setTxnId(System.currentTimeMillis()) - .setVersion(ProtocolVersion.VERSION_THREE) - .setOperation(OperationType.READ_ENTRY) - .build()) - .setReadRequest(ReadRequest.newBuilder() - .setLedgerId(10) - .setEntryId(1) - .setMasterKey(ByteString.copyFrom(new byte[0])) - .setPreviousLAC(0) - .setTimeOut(1) - .build()) - .build(); + Request request = new Request(); + request.setHeader() + .setTxnId(System.currentTimeMillis()) + .setVersion(ProtocolVersion.VERSION_THREE) + .setOperation(OperationType.READ_ENTRY); + request.setReadRequest() + .setLedgerId(10) + .setEntryId(1) + .setMasterKey(new byte[0]) + .setPreviousLAC(0) + .setTimeOut(1); Channel channel = mock(Channel.class); when(channel.isOpen()).thenReturn(true); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBackwardCompatCMS42.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBackwardCompatCMS42.java index 0c03eca7ecf..fb190d8345d 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBackwardCompatCMS42.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBackwardCompatCMS42.java @@ -24,8 +24,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import com.google.protobuf.ByteString; -import com.google.protobuf.ExtensionRegistry; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; import io.netty.channel.EventLoopGroup; @@ -45,7 +43,7 @@ import org.apache.bookkeeper.proto.BookieProtocol.ReadRequest; import org.apache.bookkeeper.proto.BookieProtocol.Request; import org.apache.bookkeeper.proto.BookieProtocol.Response; -import org.apache.bookkeeper.proto.BookkeeperProtocol.AuthMessage; +import org.apache.bookkeeper.proto.AuthMessage; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Test; @@ -63,7 +61,6 @@ public class TestBackwardCompatCMS42 extends BookKeeperClusterTestCase { private static final byte[] FAILURE_RESPONSE = {2}; private static final byte[] PAYLOAD_MESSAGE = {3}; - ExtensionRegistry extRegistry = ExtensionRegistry.newInstance(); ClientAuthProvider.Factory authProvider; EventLoopGroup eventLoopGroup = new NioEventLoopGroup(); OrderedExecutor executor = OrderedExecutor.newBuilder().numThreads(1).name("TestBackwardCompatClient") @@ -84,10 +81,9 @@ public void testAuthSingleMessage() throws Exception { TestAuth.AlwaysSucceedBookieAuthProviderFactory.class.getName()); BookieServer bookie1 = startAndStoreBookie(bookieConf); - AuthMessage.Builder builder = AuthMessage.newBuilder() - .setAuthPluginName(TestAuth.TEST_AUTH_PROVIDER_PLUGIN_NAME); - builder.setPayload(ByteString.copyFrom(PAYLOAD_MESSAGE)); - final AuthMessage authMessage = builder.build(); + final AuthMessage authMessage = new AuthMessage() + .setAuthPluginName(TestAuth.TEST_AUTH_PROVIDER_PLUGIN_NAME) + .setPayload(PAYLOAD_MESSAGE); CompatClient42 client = newCompatClient(bookie1.getBookieId()); @@ -106,10 +102,9 @@ public void testAuthMultiMessage() throws Exception { TestAuth.SucceedAfter3BookieAuthProviderFactory.class.getName()); BookieServer bookie1 = startAndStoreBookie(bookieConf); - AuthMessage.Builder builder = AuthMessage.newBuilder() - .setAuthPluginName(TestAuth.TEST_AUTH_PROVIDER_PLUGIN_NAME); - builder.setPayload(ByteString.copyFrom(PAYLOAD_MESSAGE)); - final AuthMessage authMessage = builder.build(); + final AuthMessage authMessage = new AuthMessage() + .setAuthPluginName(TestAuth.TEST_AUTH_PROVIDER_PLUGIN_NAME) + .setPayload(PAYLOAD_MESSAGE); CompatClient42 client = newCompatClient(bookie1.getBookieId()); Request request = new AuthRequest(BookieProtocol.CURRENT_PROTOCOL_VERSION, authMessage); @@ -121,7 +116,7 @@ public void testAuthMultiMessage() throws Exception { assertEquals("Should have succeeded", response.getErrorCode(), BookieProtocol.EOK); byte[] type = authResponse.getAuthMessage() - .getPayload().toByteArray(); + .getPayload(); if (i == 2) { assertArrayEquals("Should succeed after 3", type, SUCCESS_RESPONSE); @@ -139,10 +134,9 @@ public void testAuthFail() throws Exception { TestAuth.FailAfter3BookieAuthProviderFactory.class.getName()); BookieServer bookie1 = startAndStoreBookie(bookieConf); - AuthMessage.Builder builder = AuthMessage.newBuilder() - .setAuthPluginName(TestAuth.TEST_AUTH_PROVIDER_PLUGIN_NAME); - builder.setPayload(ByteString.copyFrom(PAYLOAD_MESSAGE)); - final AuthMessage authMessage = builder.build(); + final AuthMessage authMessage = new AuthMessage() + .setAuthPluginName(TestAuth.TEST_AUTH_PROVIDER_PLUGIN_NAME) + .setPayload(PAYLOAD_MESSAGE); CompatClient42 client = newCompatClient(bookie1.getBookieId()); Request request = new AuthRequest(BookieProtocol.CURRENT_PROTOCOL_VERSION, authMessage); @@ -154,7 +148,7 @@ public void testAuthFail() throws Exception { assertEquals("Should have succeeded", response.getErrorCode(), BookieProtocol.EOK); byte[] type = authResponse.getAuthMessage() - .getPayload().toByteArray(); + .getPayload(); if (i == 2) { assertArrayEquals("Should fail after 3", type, FAILURE_RESPONSE); @@ -181,7 +175,7 @@ BookieServer startAndStoreBookie(ServerConfiguration conf) throws Exception { CompatClient42 newCompatClient(BookieId addr) throws Exception { ClientConfiguration conf = new ClientConfiguration(); conf.setUseV2WireProtocol(true); - return new CompatClient42(conf, executor, eventLoopGroup, addr, authProvider, extRegistry); + return new CompatClient42(conf, executor, eventLoopGroup, addr, authProvider); } // extending PerChannelBookieClient to get the pipeline factory @@ -194,15 +188,13 @@ class CompatClient42 extends PerChannelBookieClient { OrderedExecutor executor, EventLoopGroup eventLoopGroup, BookieId addr, - ClientAuthProvider.Factory authProviderFactory, - ExtensionRegistry extRegistry) throws Exception { + ClientAuthProvider.Factory authProviderFactory) throws Exception { super(conf, executor, eventLoopGroup, addr, NullStatsLogger.INSTANCE, authProviderFactory, - extRegistry, null, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBookieRequestProcessor.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBookieRequestProcessor.java index 46304023433..5769e12e6e3 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBookieRequestProcessor.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBookieRequestProcessor.java @@ -29,7 +29,6 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import com.google.protobuf.ByteString; import io.netty.buffer.UnpooledByteBufAllocator; import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; @@ -37,14 +36,14 @@ import io.netty.channel.group.DefaultChannelGroup; import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.proto.BookkeeperProtocol.AddRequest; -import org.apache.bookkeeper.proto.BookkeeperProtocol.AddRequest.Flag; -import org.apache.bookkeeper.proto.BookkeeperProtocol.BKPacketHeader; -import org.apache.bookkeeper.proto.BookkeeperProtocol.OperationType; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ProtocolVersion; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ReadRequest; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; -import org.apache.bookkeeper.proto.BookkeeperProtocol.WriteLacRequest; +import org.apache.bookkeeper.proto.AddRequest; +import org.apache.bookkeeper.proto.AddRequest.Flag; +import org.apache.bookkeeper.proto.BKPacketHeader; +import org.apache.bookkeeper.proto.OperationType; +import org.apache.bookkeeper.proto.ProtocolVersion; +import org.apache.bookkeeper.proto.ReadRequest; +import org.apache.bookkeeper.proto.Request; +import org.apache.bookkeeper.proto.WriteLacRequest; import org.apache.bookkeeper.stats.NullStatsLogger; import org.junit.Test; @@ -92,60 +91,55 @@ conf, mock(Bookie.class), NullStatsLogger.INSTANCE, null, UnpooledByteBufAllocat @Test public void testFlagsV3() { - ReadRequest read = ReadRequest.newBuilder() - .setLedgerId(10).setEntryId(1) - .setFlag(ReadRequest.Flag.FENCE_LEDGER).build(); + ReadRequest read = new ReadRequest() + .setLedgerId(10).setEntryId(1) + .setFlag(ReadRequest.Flag.FENCE_LEDGER); assertTrue(RequestUtils.hasFlag(read, ReadRequest.Flag.FENCE_LEDGER)); assertFalse(RequestUtils.hasFlag(read, ReadRequest.Flag.ENTRY_PIGGYBACK)); - read = ReadRequest.newBuilder() - .setLedgerId(10).setEntryId(1) - .setFlag(ReadRequest.Flag.ENTRY_PIGGYBACK).build(); + read = new ReadRequest() + .setLedgerId(10).setEntryId(1) + .setFlag(ReadRequest.Flag.ENTRY_PIGGYBACK); assertFalse(RequestUtils.hasFlag(read, ReadRequest.Flag.FENCE_LEDGER)); assertTrue(RequestUtils.hasFlag(read, ReadRequest.Flag.ENTRY_PIGGYBACK)); - read = ReadRequest.newBuilder() - .setLedgerId(10).setEntryId(1) - .build(); + read = new ReadRequest() + .setLedgerId(10).setEntryId(1); assertFalse(RequestUtils.hasFlag(read, ReadRequest.Flag.FENCE_LEDGER)); assertFalse(RequestUtils.hasFlag(read, ReadRequest.Flag.ENTRY_PIGGYBACK)); - AddRequest add = AddRequest.newBuilder() - .setLedgerId(10).setEntryId(1) - .setFlag(AddRequest.Flag.RECOVERY_ADD) - .setMasterKey(ByteString.EMPTY) - .setBody(ByteString.EMPTY) - .build(); + AddRequest add = new AddRequest() + .setLedgerId(10).setEntryId(1) + .setFlag(AddRequest.Flag.RECOVERY_ADD) + .setMasterKey(new byte[0]) + .setBody(new byte[0]); assertTrue(RequestUtils.hasFlag(add, AddRequest.Flag.RECOVERY_ADD)); - add = AddRequest.newBuilder() - .setLedgerId(10).setEntryId(1) - .setMasterKey(ByteString.EMPTY) - .setBody(ByteString.EMPTY) - .build(); + add = new AddRequest() + .setLedgerId(10).setEntryId(1) + .setMasterKey(new byte[0]) + .setBody(new byte[0]); assertFalse(RequestUtils.hasFlag(add, AddRequest.Flag.RECOVERY_ADD)); - add = AddRequest.newBuilder() - .setLedgerId(10).setEntryId(1) - .setFlag(AddRequest.Flag.RECOVERY_ADD) - .setMasterKey(ByteString.EMPTY) - .setBody(ByteString.EMPTY) - .build(); + add = new AddRequest() + .setLedgerId(10).setEntryId(1) + .setFlag(AddRequest.Flag.RECOVERY_ADD) + .setMasterKey(new byte[0]) + .setBody(new byte[0]); assertTrue(RequestUtils.hasFlag(add, AddRequest.Flag.RECOVERY_ADD)); } @Test public void testToString() { - BKPacketHeader.Builder headerBuilder = BKPacketHeader.newBuilder(); - headerBuilder.setVersion(ProtocolVersion.VERSION_THREE); - headerBuilder.setOperation(OperationType.ADD_ENTRY); - headerBuilder.setTxnId(5L); - BKPacketHeader header = headerBuilder.build(); - - AddRequest addRequest = AddRequest.newBuilder().setLedgerId(10).setEntryId(1) - .setMasterKey(ByteString.copyFrom("masterKey".getBytes())) - .setBody(ByteString.copyFrom("entrydata".getBytes())).build(); - Request request = Request.newBuilder().setHeader(header).setAddRequest(addRequest).build(); + Request request = new Request(); + request.setHeader() + .setVersion(ProtocolVersion.VERSION_THREE) + .setOperation(OperationType.ADD_ENTRY) + .setTxnId(5L); + request.setAddRequest() + .setLedgerId(10).setEntryId(1) + .setMasterKey("masterKey".getBytes()) + .setBody("entrydata".getBytes()); Channel channel = mock(Channel.class); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); @@ -167,11 +161,13 @@ public void testToString() { assertFalse("writeEntryProcessorV3's toString shouldn't contain flag", toString.contains("flag")); assertFalse("writeEntryProcessorV3's toString shouldn't contain writeFlags", toString.contains("writeFlags")); - addRequest = AddRequest.newBuilder().setLedgerId(10).setEntryId(1) - .setMasterKey(ByteString.copyFrom("masterKey".getBytes())) - .setBody(ByteString.copyFrom("entrydata".getBytes())).setFlag(Flag.RECOVERY_ADD).setWriteFlags(0) - .build(); - request = Request.newBuilder().setHeader(header).setAddRequest(addRequest).build(); + request = freshHeaderRequest(); + request.setAddRequest() + .setLedgerId(10).setEntryId(1) + .setMasterKey("masterKey".getBytes()) + .setBody("entrydata".getBytes()) + .setFlag(Flag.RECOVERY_ADD) + .setWriteFlags(0); writeEntryProcessorV3 = new WriteEntryProcessorV3(request, requestHandler, requestProcessor); toString = writeEntryProcessorV3.toString(); assertFalse("writeEntryProcessorV3's toString should have filtered out body", toString.contains("body")); @@ -180,9 +176,10 @@ public void testToString() { assertTrue("writeEntryProcessorV3's toString should contain flag", toString.contains("flag")); assertTrue("writeEntryProcessorV3's toString should contain writeFlags", toString.contains("writeFlags")); - ReadRequest readRequest = ReadRequest.newBuilder().setLedgerId(10).setEntryId(23) - .setMasterKey(ByteString.copyFrom("masterKey".getBytes())).build(); - request = Request.newBuilder().setHeader(header).setReadRequest(readRequest).build(); + request = freshHeaderRequest(); + request.setReadRequest() + .setLedgerId(10).setEntryId(23) + .setMasterKey("masterKey".getBytes()); toString = RequestUtils.toSafeString(request); assertFalse("ReadRequest's safeString should have filtered out masterKey", toString.contains("masterKey")); assertTrue("ReadRequest's safeString should contain ledgerId", toString.contains("ledgerId")); @@ -194,20 +191,22 @@ public void testToString() { assertFalse("ReadRequest's safeString shouldn't contain previousLAC", toString.contains("previousLAC")); assertFalse("ReadRequest's safeString shouldn't contain timeOut", toString.contains("timeOut")); - readRequest = ReadRequest.newBuilder().setLedgerId(10).setEntryId(23).setPreviousLAC(2).setTimeOut(100) - .setMasterKey(ByteString.copyFrom("masterKey".getBytes())).setFlag(ReadRequest.Flag.ENTRY_PIGGYBACK) - .build(); - request = Request.newBuilder().setHeader(header).setReadRequest(readRequest).build(); + request = freshHeaderRequest(); + request.setReadRequest() + .setLedgerId(10).setEntryId(23).setPreviousLAC(2).setTimeOut(100) + .setMasterKey("masterKey".getBytes()) + .setFlag(ReadRequest.Flag.ENTRY_PIGGYBACK); toString = RequestUtils.toSafeString(request); assertFalse("ReadRequest's safeString should have filtered out masterKey", toString.contains("masterKey")); assertTrue("ReadRequest's safeString shouldn contain flag", toString.contains("flag")); assertTrue("ReadRequest's safeString shouldn contain previousLAC", toString.contains("previousLAC")); assertTrue("ReadRequest's safeString shouldn contain timeOut", toString.contains("timeOut")); - WriteLacRequest writeLacRequest = WriteLacRequest.newBuilder().setLedgerId(10).setLac(23) - .setMasterKey(ByteString.copyFrom("masterKey".getBytes())) - .setBody(ByteString.copyFrom("entrydata".getBytes())).build(); - request = Request.newBuilder().setHeader(header).setWriteLacRequest(writeLacRequest).build(); + request = freshHeaderRequest(); + request.setWriteLacRequest() + .setLedgerId(10).setLac(23) + .setMasterKey("masterKey".getBytes()) + .setBody("entrydata".getBytes()); WriteLacProcessorV3 writeLacProcessorV3 = new WriteLacProcessorV3(request, null, requestProcessor); toString = writeLacProcessorV3.toString(); assertFalse("writeLacProcessorV3's toString should have filtered out body", toString.contains("body")); @@ -219,4 +218,13 @@ public void testToString() { assertTrue("writeLacProcessorV3's toString should contain operation", toString.contains("operation")); assertTrue("writeLacProcessorV3's toString should contain txnId", toString.contains("txnId")); } + + private static Request freshHeaderRequest() { + Request request = new Request(); + request.setHeader() + .setVersion(ProtocolVersion.VERSION_THREE) + .setOperation(OperationType.ADD_ENTRY) + .setTxnId(5L); + return request; + } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java index b52e8b95db7..8c585748404 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java @@ -24,7 +24,6 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import com.google.protobuf.ExtensionRegistry; import io.netty.buffer.ByteBuf; import io.netty.channel.Channel; import io.netty.channel.EventLoopGroup; @@ -65,7 +64,6 @@ public class TestPerChannelBookieClient extends BookKeeperClusterTestCase { private static final Logger LOG = LoggerFactory.getLogger(TestPerChannelBookieClient.class); - ExtensionRegistry extRegistry = ExtensionRegistry.newInstance(); ClientAuthProvider.Factory authProvider; public TestPerChannelBookieClient() throws Exception { @@ -90,7 +88,7 @@ public void testConnectCloseRace() throws Exception { BookieId addr = getBookie(0); for (int i = 0; i < 1000; i++) { PerChannelBookieClient client = new PerChannelBookieClient(executor, eventLoopGroup, addr, - authProvider, extRegistry, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); + authProvider, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); client.connectIfNeededAndDoOp(new GenericCallback() { @Override public void operationComplete(int rc, PerChannelBookieClient client) { @@ -133,7 +131,7 @@ public void operationComplete(int rc, PerChannelBookieClient pcbc) { BookieId addr = getBookie(0); for (int i = 0; i < 100; i++) { PerChannelBookieClient client = new PerChannelBookieClient(executor, eventLoopGroup, addr, - authProvider, extRegistry, + authProvider, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); for (int j = i; j < 10; j++) { client.connectIfNeededAndDoOp(nullop); @@ -165,7 +163,7 @@ public void operationComplete(int rc, PerChannelBookieClient client) { BookieId addr = getBookie(0); final PerChannelBookieClient client = new PerChannelBookieClient(executor, eventLoopGroup, - addr, authProvider, extRegistry, + addr, authProvider, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); final AtomicBoolean shouldFail = new AtomicBoolean(false); final AtomicBoolean running = new AtomicBoolean(true); @@ -261,7 +259,7 @@ public ByteBuf readEntry(long ledgerId, long entryId) BookieId addr = getBookie(0); final PerChannelBookieClient client = new PerChannelBookieClient(executor, eventLoopGroup, - addr, authProvider, extRegistry, + addr, authProvider, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); final CountDownLatch completion = new CountDownLatch(1); final ReadEntryCallback cb = new ReadEntryCallback() { @@ -311,7 +309,7 @@ public void testEpollChannelTcpUserTimeout() throws Exception { // Pass to the PerChannelBookieClient object the client configuration with TCP user timeout. PerChannelBookieClient channel = new PerChannelBookieClient(conf, executor, eventLoopGroup, - addr, Mockito.mock(StatsLogger.class), authProvider, extRegistry, + addr, Mockito.mock(StatsLogger.class), authProvider, Mockito.mock(PerChannelBookieClientPool.class), BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); // Verify that the configured value has not been set in the channel if does not exist in config. @@ -322,7 +320,7 @@ public void testEpollChannelTcpUserTimeout() throws Exception { // Create a new channel with new TCP user timeout set. conf.setTcpUserTimeoutMillis(tcpUserTimeout); channel = new PerChannelBookieClient(conf, executor, eventLoopGroup, - addr, Mockito.mock(StatsLogger.class), authProvider, extRegistry, + addr, Mockito.mock(StatsLogger.class), authProvider, Mockito.mock(PerChannelBookieClientPool.class), BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); // Verify that the configured value has been set. diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3Test.java index 94024aa5c0d..ed6deeb9ad9 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3Test.java @@ -29,7 +29,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import com.google.protobuf.ByteString; import io.netty.buffer.ByteBuf; import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; @@ -40,13 +39,13 @@ import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; -import org.apache.bookkeeper.proto.BookkeeperProtocol.AddRequest; -import org.apache.bookkeeper.proto.BookkeeperProtocol.BKPacketHeader; -import org.apache.bookkeeper.proto.BookkeeperProtocol.OperationType; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ProtocolVersion; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Response; -import org.apache.bookkeeper.proto.BookkeeperProtocol.StatusCode; +import org.apache.bookkeeper.proto.AddRequest; +import org.apache.bookkeeper.proto.BKPacketHeader; +import org.apache.bookkeeper.proto.OperationType; +import org.apache.bookkeeper.proto.ProtocolVersion; +import org.apache.bookkeeper.proto.Request; +import org.apache.bookkeeper.proto.Response; +import org.apache.bookkeeper.proto.StatusCode; import org.apache.bookkeeper.stats.NullStatsLogger; import org.junit.Before; import org.junit.Test; @@ -66,19 +65,16 @@ public class WriteEntryProcessorV3Test { @Before public void setup() { - request = Request.newBuilder() - .setHeader(BKPacketHeader.newBuilder() + request = new Request(); + request.setHeader() .setTxnId(System.currentTimeMillis()) .setVersion(ProtocolVersion.VERSION_THREE) - .setOperation(OperationType.ADD_ENTRY) - .build()) - .setAddRequest(AddRequest.newBuilder() + .setOperation(OperationType.ADD_ENTRY); + request.setAddRequest() .setLedgerId(System.currentTimeMillis()) .setEntryId(System.currentTimeMillis() + 1) - .setBody(ByteString.copyFromUtf8("test-entry-data")) - .setMasterKey(ByteString.copyFrom(new byte[0])) - .build()) - .build(); + .setBody("test-entry-data".getBytes()) + .setMasterKey(new byte[0]); channel = mock(Channel.class); when(channel.isOpen()).thenReturn(true); @@ -100,11 +96,7 @@ public void setup() { } private void reinitRequest(int priority) { - request = Request.newBuilder(request) - .setHeader(BKPacketHeader.newBuilder(request.getHeader()) - .setPriority(priority) - .build()) - .build(); + request.setHeader().setPriority(priority); processor = new WriteEntryProcessorV3( request, diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java index e9928a01c86..165a8d1cad0 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java @@ -69,7 +69,7 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GetBookieInfoCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; -import org.apache.bookkeeper.proto.BookkeeperProtocol; +import org.apache.bookkeeper.proto.GetBookieInfoRequest; import org.apache.bookkeeper.proto.LedgerMetadataFormat; import org.apache.bookkeeper.proto.PerChannelBookieClient; import org.apache.bookkeeper.proto.PerChannelBookieClientPool; @@ -318,8 +318,8 @@ public void testGetBookieInfo(boolean limitStatsLogging) throws IOException, Int TestStatsLogger statsLogger = statsProvider.getStatsLogger(""); BookieClient bc = new BookieClientImpl(clientConf, new NioEventLoopGroup(), UnpooledByteBufAllocator.DEFAULT, executor, scheduler, statsLogger, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); - long flags = BookkeeperProtocol.GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE - | BookkeeperProtocol.GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE; + long flags = GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE + | GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE; class CallbackObj { int rc; @@ -341,11 +341,11 @@ public void getBookieInfoComplete(int rc, BookieInfo bInfo, Object ctx) { CallbackObj obj = (CallbackObj) ctx; obj.rc = rc; if (rc == Code.OK) { - if ((obj.requested & BookkeeperProtocol.GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE) != 0) { + if ((obj.requested & GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE) != 0) { obj.freeDiskSpace = bInfo.getFreeDiskSpace(); } if ((obj.requested - & BookkeeperProtocol.GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE) != 0) { + & GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE) != 0) { obj.totalDiskCapacity = bInfo.getTotalDiskSpace(); } } diff --git a/buildtools/src/main/resources/bookkeeper/findbugsExclude.xml b/buildtools/src/main/resources/bookkeeper/findbugsExclude.xml index 326cfb36eea..5064de54512 100644 --- a/buildtools/src/main/resources/bookkeeper/findbugsExclude.xml +++ b/buildtools/src/main/resources/bookkeeper/findbugsExclude.xml @@ -37,14 +37,9 @@ - + - - - - - - + diff --git a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/ProtocolBenchmark.java b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/ProtocolBenchmark.java index a44890845a2..bdf8476ff36 100644 --- a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/ProtocolBenchmark.java +++ b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/ProtocolBenchmark.java @@ -15,92 +15,86 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.bookkeeper.proto; -import static java.nio.charset.StandardCharsets.UTF_8; +package org.apache.bookkeeper.proto; -import com.google.protobuf.ByteString; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; -import io.netty.buffer.Unpooled; +import io.netty.buffer.PooledByteBufAllocator; import io.netty.util.ReferenceCountUtil; -import java.util.concurrent.ThreadLocalRandom; +import java.security.SecureRandom; +import java.util.Random; import java.util.concurrent.TimeUnit; -import org.apache.bookkeeper.proto.BookieProtoEncoding.EnDecoder; import org.apache.bookkeeper.proto.BookieProtoEncoding.RequestEnDeCoderPreV3; import org.apache.bookkeeper.proto.BookieProtoEncoding.RequestEnDecoderV3; -import org.apache.bookkeeper.proto.BookkeeperProtocol.AddRequest; -import org.apache.bookkeeper.proto.BookkeeperProtocol.BKPacketHeader; -import org.apache.bookkeeper.proto.BookkeeperProtocol.OperationType; -import org.apache.bookkeeper.proto.BookkeeperProtocol.ProtocolVersion; -import org.apache.bookkeeper.proto.BookkeeperProtocol.Request; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; import org.openjdk.jmh.annotations.Mode; import org.openjdk.jmh.annotations.OutputTimeUnit; -import org.openjdk.jmh.annotations.Param; import org.openjdk.jmh.annotations.Scope; import org.openjdk.jmh.annotations.Setup; import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; import org.slf4j.MDC; /** - * Benchmarking serialization and deserialization. + * Benchmark protocol encoding. */ @BenchmarkMode({Mode.Throughput}) @OutputTimeUnit(TimeUnit.MILLISECONDS) -@State(Scope.Thread) +@Warmup(iterations = 5, time = 5, timeUnit = TimeUnit.SECONDS) +@Measurement(iterations = 5, time = 10, timeUnit = TimeUnit.SECONDS) +@Fork(value = 1, jvmArgsAppend = {"-Xms2g", "-Xmx2g"}) +@State(Scope.Benchmark) public class ProtocolBenchmark { - @Param({"10", "100", "1000", "10000"}) - int size; + private final long ledgerId = System.currentTimeMillis(); + private final long entryId = ledgerId + 1L; + + private final byte[] masterKey = new byte[20]; + private ByteBuf entry; - byte[] masterKey; - ByteBuf entry; - long ledgerId; - long entryId; - short flags; - EnDecoder reqEnDeV2; - EnDecoder reqEnDeV3; + private RequestEnDeCoderPreV3 reqEnDeV2; + private RequestEnDecoderV3 reqEnDeV3; @Setup - public void prepare() { - this.masterKey = "test-benchmark-key".getBytes(UTF_8); - byte[] data = new byte[this.size]; - ThreadLocalRandom.current().nextBytes(data); - this.entry = Unpooled.wrappedBuffer(data); - this.ledgerId = ThreadLocalRandom.current().nextLong(); - this.entryId = ThreadLocalRandom.current().nextLong(); - this.flags = 1; + public void setup() { + // prepare master key + Random rng = new SecureRandom(); + rng.nextBytes(masterKey); + + // prepare entry data + byte[] entryData = new byte[1024]; + rng.nextBytes(entryData); + entry = PooledByteBufAllocator.DEFAULT.buffer(entryData.length); + entry.writeBytes(entryData); // prepare the encoder - this.reqEnDeV2 = new RequestEnDeCoderPreV3(null); - this.reqEnDeV3 = new RequestEnDecoderV3(null); + this.reqEnDeV2 = new RequestEnDeCoderPreV3(); + this.reqEnDeV3 = new RequestEnDecoderV3(); } @Benchmark public void testAddEntryV3() throws Exception { // Build the request and calculate the total size to be included in the packet. - BKPacketHeader.Builder headerBuilder = BKPacketHeader.newBuilder() - .setVersion(ProtocolVersion.VERSION_THREE) - .setOperation(OperationType.ADD_ENTRY) - .setTxnId(0L); - ByteBuf toSend = entry.slice(); byte[] toSendArray = new byte[toSend.readableBytes()]; toSend.getBytes(toSend.readerIndex(), toSendArray); - AddRequest.Builder addBuilder = AddRequest.newBuilder() + + Request request = new Request(); + request.setHeader() + .setVersion(ProtocolVersion.VERSION_THREE) + .setOperation(OperationType.ADD_ENTRY) + .setTxnId(0L); + request.setAddRequest() .setLedgerId(ledgerId) .setEntryId(entryId) - .setMasterKey(ByteString.copyFrom(masterKey)) - .setBody(ByteString.copyFrom(toSendArray)) + .setMasterKey(masterKey) + .setBody(toSendArray) .setFlag(AddRequest.Flag.RECOVERY_ADD); - Request request = Request.newBuilder() - .setHeader(headerBuilder) - .setAddRequest(addBuilder) - .build(); - Object res = this.reqEnDeV3.encode(request, ByteBufAllocator.DEFAULT); ReferenceCountUtil.release(res); } @@ -110,69 +104,57 @@ public void testAddEntryV3WithMdc() throws Exception { MDC.put("parent_id", "LetsPutSomeLongParentRequestIdHere"); MDC.put("request_id", "LetsPutSomeLongRequestIdHere"); // Build the request and calculate the total size to be included in the packet. - BKPacketHeader.Builder headerBuilder = BKPacketHeader.newBuilder() - .setVersion(ProtocolVersion.VERSION_THREE) - .setOperation(OperationType.ADD_ENTRY) - .setTxnId(0L); - ByteBuf toSend = entry.slice(); byte[] toSendArray = new byte[toSend.readableBytes()]; toSend.getBytes(toSend.readerIndex(), toSendArray); - AddRequest.Builder addBuilder = AddRequest.newBuilder() + + Request request = new Request(); + request.setHeader() + .setVersion(ProtocolVersion.VERSION_THREE) + .setOperation(OperationType.ADD_ENTRY) + .setTxnId(0L); + request.setAddRequest() .setLedgerId(ledgerId) .setEntryId(entryId) - .setMasterKey(ByteString.copyFrom(masterKey)) - .setBody(ByteString.copyFrom(toSendArray)) + .setMasterKey(masterKey) + .setBody(toSendArray) .setFlag(AddRequest.Flag.RECOVERY_ADD); - - Request request = PerChannelBookieClient.appendRequestContext(Request.newBuilder()) - .setHeader(headerBuilder) - .setAddRequest(addBuilder) - .build(); + PerChannelBookieClient.appendRequestContext(request); Object res = this.reqEnDeV3.encode(request, ByteBufAllocator.DEFAULT); ReferenceCountUtil.release(res); MDC.clear(); } - static Request.Builder appendRequestContextNoMdc(Request.Builder builder) { - final BookkeeperProtocol.ContextPair context1 = BookkeeperProtocol.ContextPair.newBuilder() + static Request appendRequestContextNoMdc(Request request) { + request.addRequestContext() .setKey("parent_id") - .setValue("LetsPutSomeLongParentRequestIdHere") - .build(); - builder.addRequestContext(context1); - - final BookkeeperProtocol.ContextPair context2 = BookkeeperProtocol.ContextPair.newBuilder() + .setValue("LetsPutSomeLongParentRequestIdHere"); + request.addRequestContext() .setKey("request_id") - .setValue("LetsPutSomeLongRequestIdHere") - .build(); - builder.addRequestContext(context2); - - return builder; + .setValue("LetsPutSomeLongRequestIdHere"); + return request; } @Benchmark public void testAddEntryV3WithExtraContextDataNoMdc() throws Exception { // Build the request and calculate the total size to be included in the packet. - BKPacketHeader.Builder headerBuilder = BKPacketHeader.newBuilder() - .setVersion(ProtocolVersion.VERSION_THREE) - .setOperation(OperationType.ADD_ENTRY) - .setTxnId(0L); - ByteBuf toSend = entry.slice(); byte[] toSendArray = new byte[toSend.readableBytes()]; toSend.getBytes(toSend.readerIndex(), toSendArray); - AddRequest.Builder addBuilder = AddRequest.newBuilder() + + Request request = new Request(); + request.setHeader() + .setVersion(ProtocolVersion.VERSION_THREE) + .setOperation(OperationType.ADD_ENTRY) + .setTxnId(0L); + request.setAddRequest() .setLedgerId(ledgerId) .setEntryId(entryId) - .setMasterKey(ByteString.copyFrom(masterKey)) - .setBody(ByteString.copyFrom(toSendArray)) + .setMasterKey(masterKey) + .setBody(toSendArray) .setFlag(AddRequest.Flag.RECOVERY_ADD); - - Request request = appendRequestContextNoMdc(Request.newBuilder()) - .setHeader(headerBuilder) - .setAddRequest(addBuilder) - .build(); + appendRequestContextNoMdc(request); Object res = this.reqEnDeV3.encode(request, ByteBufAllocator.DEFAULT); ReferenceCountUtil.release(res); From 2222dc62de795ce6244e90e27c41b4d934f3cb60 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 7 May 2026 15:14:17 -0700 Subject: [PATCH 06/11] Fix checkstyle: remove redundant same-package imports --- .../org/apache/bookkeeper/proto/AuthHandler.java | 1 - .../org/apache/bookkeeper/proto/BookieProtocol.java | 1 - .../org/apache/bookkeeper/proto/CompletionKey.java | 1 - .../bookkeeper/proto/ForceLedgerProcessorV3.java | 5 ----- .../bookkeeper/proto/GetBookieInfoProcessorV3.java | 5 ----- .../proto/GetListOfEntriesOfLedgerProcessorV3.java | 5 ----- .../proto/LongPollReadEntryProcessorV3.java | 3 --- .../bookkeeper/proto/PacketProcessorBaseV3.java | 4 ---- .../bookkeeper/proto/PerChannelBookieClient.java | 13 ------------- .../bookkeeper/proto/ReadEntryProcessorV3.java | 5 ----- .../apache/bookkeeper/proto/ReadLacProcessorV3.java | 5 ----- .../bookkeeper/proto/WriteEntryProcessorV3.java | 5 ----- .../bookkeeper/proto/WriteLacProcessorV3.java | 5 ----- .../bookkeeper/proto/BookieProtoEncodingTest.java | 3 +-- .../proto/ForceLedgerProcessorV3Test.java | 7 ------- .../proto/LongPollReadEntryProcessorV3Test.java | 5 ----- .../bookkeeper/proto/TestBackwardCompatCMS42.java | 1 - .../proto/TestBookieRequestProcessor.java | 7 ------- .../bookkeeper/proto/WriteEntryProcessorV3Test.java | 7 ------- 19 files changed, 1 insertion(+), 87 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AuthHandler.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AuthHandler.java index 4f4f6c0ba0c..5bcb44b7ef5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AuthHandler.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AuthHandler.java @@ -39,7 +39,6 @@ import org.apache.bookkeeper.auth.BookieAuthProvider; import org.apache.bookkeeper.auth.ClientAuthProvider; import org.apache.bookkeeper.client.BKException; -import org.apache.bookkeeper.proto.AuthMessage; import org.apache.bookkeeper.util.ByteBufList; import org.apache.bookkeeper.util.NettyChannelUtil; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtocol.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtocol.java index d202ccc8cc2..657d7fddb1d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtocol.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtocol.java @@ -26,7 +26,6 @@ import io.netty.util.Recycler.Handle; import io.netty.util.ReferenceCountUtil; import io.netty.util.ReferenceCounted; -import org.apache.bookkeeper.proto.AuthMessage; import org.apache.bookkeeper.util.ByteBufList; /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/CompletionKey.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/CompletionKey.java index 25c6be5ace7..1fb2564be6b 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/CompletionKey.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/CompletionKey.java @@ -21,7 +21,6 @@ package org.apache.bookkeeper.proto; -import org.apache.bookkeeper.proto.OperationType; abstract class CompletionKey { OperationType operationType; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java index dd40d45f90e..ef0fe82c8fb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3.java @@ -27,11 +27,6 @@ import org.apache.bookkeeper.bookie.BookieImpl; import org.apache.bookkeeper.common.util.MathUtils; import org.apache.bookkeeper.net.BookieId; -import org.apache.bookkeeper.proto.ForceLedgerRequest; -import org.apache.bookkeeper.proto.ForceLedgerResponse; -import org.apache.bookkeeper.proto.Request; -import org.apache.bookkeeper.proto.Response; -import org.apache.bookkeeper.proto.StatusCode; @CustomLog diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetBookieInfoProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetBookieInfoProcessorV3.java index ce051a27662..6f7a862a513 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetBookieInfoProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetBookieInfoProcessorV3.java @@ -24,11 +24,6 @@ import java.util.concurrent.TimeUnit; import lombok.CustomLog; import org.apache.bookkeeper.common.util.MathUtils; -import org.apache.bookkeeper.proto.GetBookieInfoRequest; -import org.apache.bookkeeper.proto.GetBookieInfoResponse; -import org.apache.bookkeeper.proto.Request; -import org.apache.bookkeeper.proto.Response; -import org.apache.bookkeeper.proto.StatusCode; /** * A processor class for v3 bookie metadata packets. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetListOfEntriesOfLedgerProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetListOfEntriesOfLedgerProcessorV3.java index 7eef5935617..d8185838003 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetListOfEntriesOfLedgerProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/GetListOfEntriesOfLedgerProcessorV3.java @@ -25,11 +25,6 @@ import lombok.CustomLog; import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.common.util.MathUtils; -import org.apache.bookkeeper.proto.GetListOfEntriesOfLedgerRequest; -import org.apache.bookkeeper.proto.GetListOfEntriesOfLedgerResponse; -import org.apache.bookkeeper.proto.Request; -import org.apache.bookkeeper.proto.Response; -import org.apache.bookkeeper.proto.StatusCode; import org.apache.bookkeeper.util.AvailabilityOfEntriesOfLedger; /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java index 73d824c5b45..7e886b6e066 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java @@ -31,9 +31,6 @@ import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.LastAddConfirmedUpdateNotification; import org.apache.bookkeeper.common.util.Watcher; -import org.apache.bookkeeper.proto.ReadResponse; -import org.apache.bookkeeper.proto.Request; -import org.apache.bookkeeper.proto.StatusCode; /** * Processor handling long poll read entry request. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PacketProcessorBaseV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PacketProcessorBaseV3.java index acff521cea4..7d234c5ad3c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PacketProcessorBaseV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PacketProcessorBaseV3.java @@ -26,10 +26,6 @@ import java.util.concurrent.TimeUnit; import lombok.CustomLog; import org.apache.bookkeeper.common.util.MathUtils; -import org.apache.bookkeeper.proto.BKPacketHeader; -import org.apache.bookkeeper.proto.ProtocolVersion; -import org.apache.bookkeeper.proto.Request; -import org.apache.bookkeeper.proto.StatusCode; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.util.StringUtils; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java index b2a90187d43..7d8124bc3c7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java @@ -103,19 +103,6 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadLacCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteLacCallback; -import org.apache.bookkeeper.proto.AddRequest; -import org.apache.bookkeeper.proto.BKPacketHeader; -import org.apache.bookkeeper.proto.ForceLedgerRequest; -import org.apache.bookkeeper.proto.GetBookieInfoRequest; -import org.apache.bookkeeper.proto.GetListOfEntriesOfLedgerRequest; -import org.apache.bookkeeper.proto.OperationType; -import org.apache.bookkeeper.proto.ProtocolVersion; -import org.apache.bookkeeper.proto.ReadLacRequest; -import org.apache.bookkeeper.proto.ReadRequest; -import org.apache.bookkeeper.proto.Request; -import org.apache.bookkeeper.proto.Response; -import org.apache.bookkeeper.proto.StatusCode; -import org.apache.bookkeeper.proto.WriteLacRequest; import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.OpStatsLogger; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java index 3e01ac6bafe..1bf99cbd0d6 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java @@ -31,11 +31,6 @@ import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.common.concurrent.FutureEventListener; import org.apache.bookkeeper.common.util.MathUtils; -import org.apache.bookkeeper.proto.ReadRequest; -import org.apache.bookkeeper.proto.ReadResponse; -import org.apache.bookkeeper.proto.Request; -import org.apache.bookkeeper.proto.Response; -import org.apache.bookkeeper.proto.StatusCode; import org.apache.bookkeeper.stats.OpStatsLogger; @CustomLog diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacProcessorV3.java index 04e70d329e8..58e345a82e4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacProcessorV3.java @@ -29,11 +29,6 @@ import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.common.util.MathUtils; -import org.apache.bookkeeper.proto.ReadLacRequest; -import org.apache.bookkeeper.proto.ReadLacResponse; -import org.apache.bookkeeper.proto.Request; -import org.apache.bookkeeper.proto.Response; -import org.apache.bookkeeper.proto.StatusCode; /** * A read processor for v3 last add confirmed messages. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3.java index 8ccc76e4c24..7f3116c9d03 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3.java @@ -30,11 +30,6 @@ import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.common.util.MathUtils; import org.apache.bookkeeper.net.BookieId; -import org.apache.bookkeeper.proto.AddRequest; -import org.apache.bookkeeper.proto.AddResponse; -import org.apache.bookkeeper.proto.Request; -import org.apache.bookkeeper.proto.Response; -import org.apache.bookkeeper.proto.StatusCode; import org.apache.bookkeeper.stats.OpStatsLogger; @CustomLog diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java index 9fcfa1f44dd..4da2129f874 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacProcessorV3.java @@ -27,11 +27,6 @@ import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.common.util.MathUtils; import org.apache.bookkeeper.net.BookieId; -import org.apache.bookkeeper.proto.Request; -import org.apache.bookkeeper.proto.Response; -import org.apache.bookkeeper.proto.StatusCode; -import org.apache.bookkeeper.proto.WriteLacRequest; -import org.apache.bookkeeper.proto.WriteLacResponse; @CustomLog diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieProtoEncodingTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieProtoEncodingTest.java index d628d279aa5..ad0376dca60 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieProtoEncodingTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieProtoEncodingTest.java @@ -37,7 +37,6 @@ import org.apache.bookkeeper.proto.BookieProtoEncoding.ResponseEnDeCoderPreV3; import org.apache.bookkeeper.proto.BookieProtoEncoding.ResponseEnDecoderV3; import org.apache.bookkeeper.proto.BookieProtocol.AddResponse; -import org.apache.bookkeeper.proto.AddRequest.Flag; import org.apache.bookkeeper.util.ByteBufList; import org.junit.Test; @@ -106,7 +105,7 @@ public void testV2RequestDecoderThrowExceptionOnUnknownRequests() throws Excepti .setLedgerId(1L) .setEntryId(2L) .setMasterKey("".getBytes(UTF_8)) - .setFlag(Flag.RECOVERY_ADD) + .setFlag(AddRequest.Flag.RECOVERY_ADD) .setBody("test".getBytes(UTF_8)); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3Test.java index fd873e77cf6..5077d7cf33e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ForceLedgerProcessorV3Test.java @@ -38,13 +38,6 @@ import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieImpl; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; -import org.apache.bookkeeper.proto.BKPacketHeader; -import org.apache.bookkeeper.proto.ForceLedgerRequest; -import org.apache.bookkeeper.proto.OperationType; -import org.apache.bookkeeper.proto.ProtocolVersion; -import org.apache.bookkeeper.proto.Request; -import org.apache.bookkeeper.proto.Response; -import org.apache.bookkeeper.proto.StatusCode; import org.apache.bookkeeper.stats.NullStatsLogger; import org.junit.Before; import org.junit.Test; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3Test.java index fd6ddda0d26..0c09bec35e6 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3Test.java @@ -33,11 +33,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.bookie.Bookie; -import org.apache.bookkeeper.proto.BKPacketHeader; -import org.apache.bookkeeper.proto.OperationType; -import org.apache.bookkeeper.proto.ProtocolVersion; -import org.apache.bookkeeper.proto.ReadRequest; -import org.apache.bookkeeper.proto.Request; import org.apache.bookkeeper.stats.NullStatsLogger; import org.junit.After; import org.junit.Before; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBackwardCompatCMS42.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBackwardCompatCMS42.java index fb190d8345d..f10ee747b21 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBackwardCompatCMS42.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBackwardCompatCMS42.java @@ -43,7 +43,6 @@ import org.apache.bookkeeper.proto.BookieProtocol.ReadRequest; import org.apache.bookkeeper.proto.BookieProtocol.Request; import org.apache.bookkeeper.proto.BookieProtocol.Response; -import org.apache.bookkeeper.proto.AuthMessage; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Test; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBookieRequestProcessor.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBookieRequestProcessor.java index 5769e12e6e3..ae85849ff2e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBookieRequestProcessor.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestBookieRequestProcessor.java @@ -36,14 +36,7 @@ import io.netty.channel.group.DefaultChannelGroup; import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.proto.AddRequest; import org.apache.bookkeeper.proto.AddRequest.Flag; -import org.apache.bookkeeper.proto.BKPacketHeader; -import org.apache.bookkeeper.proto.OperationType; -import org.apache.bookkeeper.proto.ProtocolVersion; -import org.apache.bookkeeper.proto.ReadRequest; -import org.apache.bookkeeper.proto.Request; -import org.apache.bookkeeper.proto.WriteLacRequest; import org.apache.bookkeeper.stats.NullStatsLogger; import org.junit.Test; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3Test.java index ed6deeb9ad9..ec7a8ef2d4c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/WriteEntryProcessorV3Test.java @@ -39,13 +39,6 @@ import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; -import org.apache.bookkeeper.proto.AddRequest; -import org.apache.bookkeeper.proto.BKPacketHeader; -import org.apache.bookkeeper.proto.OperationType; -import org.apache.bookkeeper.proto.ProtocolVersion; -import org.apache.bookkeeper.proto.Request; -import org.apache.bookkeeper.proto.Response; -import org.apache.bookkeeper.proto.StatusCode; import org.apache.bookkeeper.stats.NullStatsLogger; import org.junit.Before; import org.junit.Test; From d17e131556970b440848e237ffddd21209a846fd Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Thu, 7 May 2026 17:38:50 -0700 Subject: [PATCH 07/11] Fix shaded-jar tests after protobuf-java removal 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 from the three shade plugin configs (bookkeeper-server-shaded, bookkeeper-server-tests-shaded, distributedlog-core-shaded). --- shaded/bookkeeper-server-shaded/pom.xml | 1 - shaded/bookkeeper-server-tests-shaded/pom.xml | 1 - shaded/distributedlog-core-shaded/pom.xml | 1 - .../tests/shaded/BookKeeperServerShadedJarTest.java | 10 +++------- .../shaded/DistributedLogCoreShadedJarTest.java | 12 ++++-------- 5 files changed, 7 insertions(+), 18 deletions(-) diff --git a/shaded/bookkeeper-server-shaded/pom.xml b/shaded/bookkeeper-server-shaded/pom.xml index 89e22a81baa..b390ee0e7ed 100644 --- a/shaded/bookkeeper-server-shaded/pom.xml +++ b/shaded/bookkeeper-server-shaded/pom.xml @@ -72,7 +72,6 @@ com.google.guava:guava com.google.guava:failureaccess - com.google.protobuf:protobuf-java org.apache.bookkeeper:bookkeeper-common org.apache.bookkeeper:bookkeeper-common-allocator org.apache.bookkeeper:native-library-common diff --git a/shaded/bookkeeper-server-tests-shaded/pom.xml b/shaded/bookkeeper-server-tests-shaded/pom.xml index 9871bb152ee..29b3444fd5e 100644 --- a/shaded/bookkeeper-server-tests-shaded/pom.xml +++ b/shaded/bookkeeper-server-tests-shaded/pom.xml @@ -96,7 +96,6 @@ com.google.guava:guava - com.google.protobuf:protobuf-java org.apache.bookkeeper:bookkeeper-server:test-jar:tests diff --git a/shaded/distributedlog-core-shaded/pom.xml b/shaded/distributedlog-core-shaded/pom.xml index cd637147ad3..bbdb92cfbaa 100644 --- a/shaded/distributedlog-core-shaded/pom.xml +++ b/shaded/distributedlog-core-shaded/pom.xml @@ -73,7 +73,6 @@ com.fasterxml.jackson.core:jackson-annotations com.google.guava:failureaccess com.google.guava:guava - com.google.protobuf:protobuf-java at.yawk.lz4:lz4-java org.apache.bookkeeper:bookkeeper-common org.apache.bookkeeper:bookkeeper-common-allocator diff --git a/tests/shaded/bookkeeper-server-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/BookKeeperServerShadedJarTest.java b/tests/shaded/bookkeeper-server-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/BookKeeperServerShadedJarTest.java index eba29f14a47..64462f9a797 100644 --- a/tests/shaded/bookkeeper-server-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/BookKeeperServerShadedJarTest.java +++ b/tests/shaded/bookkeeper-server-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/BookKeeperServerShadedJarTest.java @@ -28,15 +28,11 @@ public class BookKeeperServerShadedJarTest { @Test(expected = ClassNotFoundException.class) - public void testProtobufIsShaded() throws Exception { + public void testProtobufIsNotIncluded() throws Exception { + // protobuf-java is no longer a runtime dependency after the LightProto migration Class.forName("com.google.protobuf.Message"); } - @Test - public void testProtobufShadedPath() throws Exception { - Class.forName("org.apache.bookkeeper.shaded.com.google.protobuf.Message"); - } - @Test(expected = ClassNotFoundException.class) public void testGuavaIsShaded() throws Exception { Class.forName("com.google.common.cache.Cache"); @@ -56,7 +52,7 @@ public void testBookKeeperCommon() throws Exception { @Test public void testBookKeeperProto() throws Exception { - Class.forName("org.apache.bookkeeper.proto.BookkeeperProtocol"); + Class.forName("org.apache.bookkeeper.proto.AddRequest"); assertTrue(true); } diff --git a/tests/shaded/distributedlog-core-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/DistributedLogCoreShadedJarTest.java b/tests/shaded/distributedlog-core-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/DistributedLogCoreShadedJarTest.java index 21463070a5c..5eccb1d484b 100644 --- a/tests/shaded/distributedlog-core-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/DistributedLogCoreShadedJarTest.java +++ b/tests/shaded/distributedlog-core-shaded-test/src/test/java/org/apache/bookkeeper/tests/shaded/DistributedLogCoreShadedJarTest.java @@ -53,15 +53,11 @@ public class DistributedLogCoreShadedJarTest { @Test(expected = ClassNotFoundException.class) - public void testProtobufIsShaded() throws Exception { + public void testProtobufIsNotIncluded() throws Exception { + // protobuf-java is no longer a runtime dependency after the LightProto migration Class.forName("com.google.protobuf.Message"); } - @Test - public void testProtobufShadedPath() throws Exception { - Class.forName("dlshade.com.google.protobuf.Message"); - } - @Test(expected = ClassNotFoundException.class) public void testGuavaIsShaded() throws Exception { Class.forName("com.google.common.cache.Cache"); @@ -97,12 +93,12 @@ public void testBookKeeperCommonShade() throws Exception { @Test(expected = ClassNotFoundException.class) public void testBookKeeperProto() throws Exception { - Class.forName("org.apache.bookkeeper.proto.BookkeeperProtocol"); + Class.forName("org.apache.bookkeeper.proto.AddRequest"); } @Test public void testBookKeeperProtoShade() throws Exception { - Class.forName("dlshade.org.apache.bookkeeper.proto.BookkeeperProtocol"); + Class.forName("dlshade.org.apache.bookkeeper.proto.AddRequest"); assertTrue(true); } From f668885de50d003878fcbd4c9a18cbbc7b15d698 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Fri, 8 May 2026 10:27:16 -0700 Subject: [PATCH 08/11] Use ByteBufList.toByteBuf to avoid copying request bodies 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. --- .../proto/PerChannelBookieClient.java | 5 +- .../apache/bookkeeper/util/ByteBufList.java | 46 ++++++++++++++ .../bookkeeper/util/ByteBufListTest.java | 60 +++++++++++++++++++ 3 files changed, 109 insertions(+), 2 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java index 7d8124bc3c7..e6380bff599 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java @@ -23,6 +23,7 @@ import io.netty.bootstrap.Bootstrap; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.PooledByteBufAllocator; import io.netty.buffer.UnpooledByteBufAllocator; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; @@ -695,7 +696,7 @@ void writeLac(final long ledgerId, final byte[] masterKey, final long lac, ByteB .setVersion(ProtocolVersion.VERSION_THREE) .setOperation(OperationType.WRITE_LAC) .setTxnId(txnId); - ByteBuf body = ByteBufList.coalesce(toSend); + ByteBuf body = toSend.toByteBuf(PooledByteBufAllocator.DEFAULT); Runnable releaseBody = body::release; writeLacRequest.setWriteLacRequest() .setLedgerId(ledgerId) @@ -794,7 +795,7 @@ void addEntry(final long ledgerId, byte[] masterKey, final long entryId, Referen } ByteBufList bufToSend = (ByteBufList) toSend; - ByteBuf body = ByteBufList.coalesce(bufToSend); + ByteBuf body = bufToSend.toByteBuf(PooledByteBufAllocator.DEFAULT); cleanupActionFailedBeforeWrite = body::release; cleanupActionAfterWrite = cleanupActionFailedBeforeWrite; AddRequest addRequestMsg = addEntryRequest.setAddRequest() diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ByteBufList.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ByteBufList.java index 133a37f0379..e0133fcaf36 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ByteBufList.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ByteBufList.java @@ -22,6 +22,8 @@ import com.google.common.annotations.VisibleForTesting; import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.Unpooled; import io.netty.channel.ChannelHandler.Sharable; import io.netty.channel.ChannelHandlerContext; @@ -246,6 +248,50 @@ public static ByteBuf coalesce(ByteBufList list) { return res; } + /** + * Returns a coalesced {@link ByteBuf} containing all buffers in this list, and + * decrements this {@code ByteBufList}'s reference count. + * + *

Ownership semantics: the returned {@link ByteBuf} owns a reference to the + * underlying buffer data. The caller is responsible for releasing the returned + * {@link ByteBuf} when it is no longer needed. This {@code ByteBufList} is + * released as part of this call. + * + *

If this list is empty, {@link Unpooled#EMPTY_BUFFER} is returned. + * If it contains exactly one buffer, that buffer is returned directly (with its + * reference count transferred to the caller) to avoid the overhead of wrapping + * it in a {@link CompositeByteBuf}. + * + * @param allocator the {@link ByteBufAllocator} used to allocate the + * {@link CompositeByteBuf} when more than one buffer is present + * @return a {@link ByteBuf} containing the coalesced contents of this list + */ + public ByteBuf toByteBuf(ByteBufAllocator allocator) { + final int size = buffers.size(); + if (size == 0) { + release(); + return Unpooled.EMPTY_BUFFER; + } + if (size == 1) { + // Fast path: avoid wrapping a single buffer in a CompositeByteBuf. + // Retain so the buffer survives our release() below; ownership is + // transferred to the caller. + ByteBuf single = buffers.get(0).retain(); + release(); + return single; + } + + CompositeByteBuf composite = allocator.compositeBuffer(size); + for (int i = 0; i < size; i++) { + // Buffers need to be retained because ownership is handed over to + // the CompositeByteBuf, while this ByteBufList continues to hold + // its own references until release() below. + composite.addComponent(true, buffers.get(i).retain()); + } + release(); + return composite; + } + @Override public ByteBufList retain() { super.retain(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/ByteBufListTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/ByteBufListTest.java index 822d4a7c48e..7b220d8893b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/ByteBufListTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/ByteBufListTest.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertSame; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -162,6 +163,65 @@ public void testCoalesce() throws Exception { assertEquals(Unpooled.wrappedBuffer("helloworld".getBytes()), ByteBufList.coalesce(buf)); } + @Test + public void testToByteBufEmpty() throws Exception { + ByteBufList buf = ByteBufList.get(); + assertEquals(1, buf.refCnt()); + + ByteBuf result = buf.toByteBuf(PooledByteBufAllocator.DEFAULT); + + // empty list returns the EMPTY_BUFFER sentinel and releases the list + assertEquals(Unpooled.EMPTY_BUFFER, result); + assertEquals(0, buf.refCnt()); + } + + @Test + public void testToByteBufSingle() throws Exception { + ByteBuf b1 = PooledByteBufAllocator.DEFAULT.heapBuffer(128, 128); + b1.writerIndex(b1.capacity()); + ByteBufList buf = ByteBufList.get(b1); + + assertEquals(1, buf.refCnt()); + assertEquals(1, b1.refCnt()); + + ByteBuf result = buf.toByteBuf(PooledByteBufAllocator.DEFAULT); + + // fast path: the single underlying buffer is returned directly with its + // ref count transferred to the caller; the list itself is released. + assertSame(b1, result); + assertEquals(0, buf.refCnt()); + assertEquals(1, result.refCnt()); + assertEquals(128, result.readableBytes()); + + result.release(); + assertEquals(0, b1.refCnt()); + } + + @Test + public void testToByteBufMultiple() throws Exception { + ByteBuf b1 = Unpooled.wrappedBuffer("hello".getBytes()); + ByteBuf b2 = Unpooled.wrappedBuffer("world".getBytes()); + ByteBufList buf = ByteBufList.get(b1, b2); + + assertEquals(1, buf.refCnt()); + assertEquals(1, b1.refCnt()); + assertEquals(1, b2.refCnt()); + + ByteBuf result = buf.toByteBuf(PooledByteBufAllocator.DEFAULT); + + // multi-buffer path: a CompositeByteBuf is returned whose components + // own the references previously held by the list. + assertEquals(0, buf.refCnt()); + assertEquals(1, result.refCnt()); + assertEquals(1, b1.refCnt()); + assertEquals(1, b2.refCnt()); + assertEquals(Unpooled.wrappedBuffer("helloworld".getBytes()), result); + + result.release(); + assertEquals(0, b1.refCnt()); + assertEquals(0, b2.refCnt()); + } + @Test public void testRetain() throws Exception { ByteBuf b1 = PooledByteBufAllocator.DEFAULT.heapBuffer(128, 128); From 9544598d04b7121d596745a03f56d9cd3e18381a Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Fri, 8 May 2026 11:22:44 -0700 Subject: [PATCH 09/11] Fix ByteBufList.toByteBuf to not release the source list The previous implementation released the source ByteBufList when producing the wrapping ByteBuf, but callers in PerChannelBookieClient only wrap a buffer they don't own — 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. --- .../apache/bookkeeper/util/ByteBufList.java | 39 +++++++++---------- .../bookkeeper/util/ByteBufListTest.java | 29 ++++++++------ 2 files changed, 35 insertions(+), 33 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ByteBufList.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ByteBufList.java index e0133fcaf36..3e7440b7974 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ByteBufList.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/ByteBufList.java @@ -249,46 +249,43 @@ public static ByteBuf coalesce(ByteBufList list) { } /** - * Returns a coalesced {@link ByteBuf} containing all buffers in this list, and - * decrements this {@code ByteBufList}'s reference count. + * Returns a {@link ByteBuf} that views the contents of this {@code ByteBufList} + * without copying the bytes. * - *

Ownership semantics: the returned {@link ByteBuf} owns a reference to the - * underlying buffer data. The caller is responsible for releasing the returned - * {@link ByteBuf} when it is no longer needed. This {@code ByteBufList} is - * released as part of this call. + *

Ownership semantics: this {@code ByteBufList}'s reference count is + * not changed by this call — the returned {@link ByteBuf} holds + * its own retained references to the underlying buffers. The caller is + * responsible for releasing the returned {@link ByteBuf} when it is no longer + * needed; this {@code ByteBufList} continues to be owned by its caller and + * must be released independently. * *

If this list is empty, {@link Unpooled#EMPTY_BUFFER} is returned. - * If it contains exactly one buffer, that buffer is returned directly (with its - * reference count transferred to the caller) to avoid the overhead of wrapping - * it in a {@link CompositeByteBuf}. + * If it contains exactly one buffer, a retained duplicate of that buffer is + * returned to avoid the overhead of wrapping it in a + * {@link CompositeByteBuf}. * * @param allocator the {@link ByteBufAllocator} used to allocate the * {@link CompositeByteBuf} when more than one buffer is present - * @return a {@link ByteBuf} containing the coalesced contents of this list + * @return a {@link ByteBuf} viewing the contents of this list */ public ByteBuf toByteBuf(ByteBufAllocator allocator) { final int size = buffers.size(); if (size == 0) { - release(); return Unpooled.EMPTY_BUFFER; } if (size == 1) { // Fast path: avoid wrapping a single buffer in a CompositeByteBuf. - // Retain so the buffer survives our release() below; ownership is - // transferred to the caller. - ByteBuf single = buffers.get(0).retain(); - release(); - return single; + // retainedDuplicate() gives the caller an independent ref count and + // reader/writer indexes. + return buffers.get(0).retainedDuplicate(); } CompositeByteBuf composite = allocator.compositeBuffer(size); for (int i = 0; i < size; i++) { - // Buffers need to be retained because ownership is handed over to - // the CompositeByteBuf, while this ByteBufList continues to hold - // its own references until release() below. - composite.addComponent(true, buffers.get(i).retain()); + // Retain so that the composite owns its own reference to each buffer, + // independent of this ByteBufList's lifecycle. + composite.addComponent(true, buffers.get(i).retainedDuplicate()); } - release(); return composite; } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/ByteBufListTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/ByteBufListTest.java index 7b220d8893b..0e88fbfac65 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/ByteBufListTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/util/ByteBufListTest.java @@ -20,7 +20,6 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertSame; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -170,9 +169,11 @@ public void testToByteBufEmpty() throws Exception { ByteBuf result = buf.toByteBuf(PooledByteBufAllocator.DEFAULT); - // empty list returns the EMPTY_BUFFER sentinel and releases the list + // empty list returns the EMPTY_BUFFER sentinel and the list is untouched assertEquals(Unpooled.EMPTY_BUFFER, result); - assertEquals(0, buf.refCnt()); + assertEquals(1, buf.refCnt()); + + buf.release(); } @Test @@ -186,14 +187,15 @@ public void testToByteBufSingle() throws Exception { ByteBuf result = buf.toByteBuf(PooledByteBufAllocator.DEFAULT); - // fast path: the single underlying buffer is returned directly with its - // ref count transferred to the caller; the list itself is released. - assertSame(b1, result); - assertEquals(0, buf.refCnt()); - assertEquals(1, result.refCnt()); + // fast path: a retained duplicate of the single buffer is returned with + // its own independent ref count; the list itself is unchanged. + assertEquals(1, buf.refCnt()); + assertEquals(2, b1.refCnt()); assertEquals(128, result.readableBytes()); result.release(); + assertEquals(1, b1.refCnt()); + buf.release(); assertEquals(0, b1.refCnt()); } @@ -210,14 +212,17 @@ public void testToByteBufMultiple() throws Exception { ByteBuf result = buf.toByteBuf(PooledByteBufAllocator.DEFAULT); // multi-buffer path: a CompositeByteBuf is returned whose components - // own the references previously held by the list. - assertEquals(0, buf.refCnt()); + // hold their own retained references; the list itself is unchanged. + assertEquals(1, buf.refCnt()); assertEquals(1, result.refCnt()); - assertEquals(1, b1.refCnt()); - assertEquals(1, b2.refCnt()); + assertEquals(2, b1.refCnt()); + assertEquals(2, b2.refCnt()); assertEquals(Unpooled.wrappedBuffer("helloworld".getBytes()), result); result.release(); + assertEquals(1, b1.refCnt()); + assertEquals(1, b2.refCnt()); + buf.release(); assertEquals(0, b1.refCnt()); assertEquals(0, b2.refCnt()); } From 76f5d0dddc4c1b70e3b9bf5ac6b15485c5ece79e Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Fri, 8 May 2026 12:33:27 -0700 Subject: [PATCH 10/11] Don't read required ledgerId/entryId from error responses 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. --- .../bookkeeper/proto/AddCompletion.java | 15 ++++++--- .../proto/ForceLedgerCompletion.java | 12 ++++--- .../bookkeeper/proto/ReadCompletion.java | 32 +++++++++++-------- .../bookkeeper/proto/WriteLacCompletion.java | 12 ++++--- 4 files changed, 44 insertions(+), 27 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AddCompletion.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AddCompletion.java index 83f99235eb6..1ed31b8dd09 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AddCompletion.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/AddCompletion.java @@ -127,11 +127,16 @@ public void handleV2Response( public void handleV3Response( Response response) { perChannelBookieClient.addEntryOutstanding.dec(); - AddResponse addResponse = response.getAddResponse(); - StatusCode status = response.getStatus() == StatusCode.EOK - ? addResponse.getStatus() : response.getStatus(); - handleResponse(addResponse.getLedgerId(), addResponse.getEntryId(), - status); + StatusCode status; + if (response.getStatus() == StatusCode.EOK && response.hasAddResponse()) { + status = response.getAddResponse().getStatus(); + } else { + // Error responses (e.g. EUA from a rejected auth handshake) may not + // carry an AddResponse with ledgerId/entryId populated. Fall back to + // the values we recorded from the outgoing request. + status = response.getStatus(); + } + handleResponse(ledgerId, entryId, status); } private void handleResponse(long ledgerId, long entryId, diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerCompletion.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerCompletion.java index 6e52d087616..34bd0384fb9 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerCompletion.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ForceLedgerCompletion.java @@ -56,10 +56,14 @@ public void errorOut(final int rc) { @Override public void handleV3Response(Response response) { - ForceLedgerResponse forceLedgerResponse = response.getForceLedgerResponse(); - StatusCode status = response.getStatus() == StatusCode.EOK - ? forceLedgerResponse.getStatus() : response.getStatus(); - long ledgerId = forceLedgerResponse.getLedgerId(); + StatusCode status; + if (response.getStatus() == StatusCode.EOK && response.hasForceLedgerResponse()) { + status = response.getForceLedgerResponse().getStatus(); + } else { + // Error responses may not carry a populated ForceLedgerResponse; + // fall back to the request's recorded ledgerId. + status = response.getStatus(); + } logEvent(status).log("Got response from bookie"); int rc = convertStatus(status, BKException.Code.WriteException); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadCompletion.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadCompletion.java index bde804756ac..10a389d5dcd 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadCompletion.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadCompletion.java @@ -81,24 +81,28 @@ public void handleV2Response(long ledgerId, long entryId, @Override public void handleV3Response(Response response) { perChannelBookieClient.readEntryOutstanding.dec(); - ReadResponse readResponse = response.getReadResponse(); - StatusCode status = response.getStatus() == StatusCode.EOK - ? readResponse.getStatus() : response.getStatus(); + StatusCode status; ByteBuf buffer = Unpooled.EMPTY_BUFFER; - if (readResponse.hasBody()) { - buffer = readResponse.getBodySlice(); - } long maxLAC = INVALID_ENTRY_ID; - if (readResponse.hasMaxLAC()) { - maxLAC = readResponse.getMaxLAC(); - } long lacUpdateTimestamp = -1L; - if (readResponse.hasLacUpdateTimestamp()) { - lacUpdateTimestamp = readResponse.getLacUpdateTimestamp(); + if (response.getStatus() == StatusCode.EOK && response.hasReadResponse()) { + ReadResponse readResponse = response.getReadResponse(); + status = readResponse.getStatus(); + if (readResponse.hasBody()) { + buffer = readResponse.getBodySlice(); + } + if (readResponse.hasMaxLAC()) { + maxLAC = readResponse.getMaxLAC(); + } + if (readResponse.hasLacUpdateTimestamp()) { + lacUpdateTimestamp = readResponse.getLacUpdateTimestamp(); + } + } else { + // Error responses may not carry a populated ReadResponse; + // fall back to the request's recorded ledgerId/entryId. + status = response.getStatus(); } - handleReadResponse(readResponse.getLedgerId(), - readResponse.getEntryId(), - status, buffer, maxLAC, lacUpdateTimestamp); + handleReadResponse(ledgerId, entryId, status, buffer, maxLAC, lacUpdateTimestamp); ReferenceCountUtil.release( buffer); // meaningless using unpooled, but client may expect to hold the last reference } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacCompletion.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacCompletion.java index 599ab36c6e4..ffbefb0f9dd 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacCompletion.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/WriteLacCompletion.java @@ -62,10 +62,14 @@ public void errorOut(final int rc) { @Override public void handleV3Response(Response response) { - WriteLacResponse writeLacResponse = response.getWriteLacResponse(); - StatusCode status = response.getStatus() == StatusCode.EOK - ? writeLacResponse.getStatus() : response.getStatus(); - long ledgerId = writeLacResponse.getLedgerId(); + StatusCode status; + if (response.getStatus() == StatusCode.EOK && response.hasWriteLacResponse()) { + status = response.getWriteLacResponse().getStatus(); + } else { + // Error responses may not carry a populated WriteLacResponse; + // fall back to the request's recorded ledgerId. + status = response.getStatus(); + } logEvent(status).log("Got response from bookie"); int rc = convertStatus(status, BKException.Code.WriteException); From e0bbc56fe506ab9532f6ed2ef4958a3ec0876cb5 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Fri, 8 May 2026 14:45:03 -0700 Subject: [PATCH 11/11] Read response ledgerId/entryId on long-poll reads in ReadCompletion 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. --- .../org/apache/bookkeeper/proto/ReadCompletion.java | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadCompletion.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadCompletion.java index 10a389d5dcd..828bfccf577 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadCompletion.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadCompletion.java @@ -81,6 +81,8 @@ public void handleV2Response(long ledgerId, long entryId, @Override public void handleV3Response(Response response) { perChannelBookieClient.readEntryOutstanding.dec(); + long respLedgerId = ledgerId; + long respEntryId = entryId; StatusCode status; ByteBuf buffer = Unpooled.EMPTY_BUFFER; long maxLAC = INVALID_ENTRY_ID; @@ -88,6 +90,14 @@ public void handleV3Response(Response response) { if (response.getStatus() == StatusCode.EOK && response.hasReadResponse()) { ReadResponse readResponse = response.getReadResponse(); status = readResponse.getStatus(); + // For long-poll reads the request entryId is LAST_ADD_CONFIRMED + // and the server fills in the actual entry id alongside the body. + if (readResponse.hasLedgerId()) { + respLedgerId = readResponse.getLedgerId(); + } + if (readResponse.hasEntryId()) { + respEntryId = readResponse.getEntryId(); + } if (readResponse.hasBody()) { buffer = readResponse.getBodySlice(); } @@ -102,7 +112,7 @@ public void handleV3Response(Response response) { // fall back to the request's recorded ledgerId/entryId. status = response.getStatus(); } - handleReadResponse(ledgerId, entryId, status, buffer, maxLAC, lacUpdateTimestamp); + handleReadResponse(respLedgerId, respEntryId, status, buffer, maxLAC, lacUpdateTimestamp); ReferenceCountUtil.release( buffer); // meaningless using unpooled, but client may expect to hold the last reference }