diff --git a/.github/workflows/bk-ci.yml b/.github/workflows/bk-ci.yml index a274f3a01ef..e8d77821147 100644 --- a/.github/workflows/bk-ci.yml +++ b/.github/workflows/bk-ci.yml @@ -116,13 +116,13 @@ jobs: module: bookkeeper-server flag: client test_args: "-Dtest='org.apache.bookkeeper.client.**'" - - step_name: Remaining Tests - module: bookkeeper-server - flag: remaining - test_args: "-Dtest='org.apache.bookkeeper.replication.**'" - step_name: Replication Tests module: bookkeeper-server flag: replication + test_args: "-Dtest='org.apache.bookkeeper.replication.**'" + - step_name: Remaining Tests + module: bookkeeper-server + flag: remaining test_args: "-Dtest='!org.apache.bookkeeper.client.**,!org.apache.bookkeeper.bookie.**,!org.apache.bookkeeper.replication.**,!org.apache.bookkeeper.tls.**'" - step_name: TLS Tests module: bookkeeper-server @@ -178,13 +178,7 @@ jobs: if [[ ! -z "${{ matrix.module }}" ]]; then projects_list="-pl ${{ matrix.module }}" fi - mvn -Pcode-coverage -B -nsu $projects_list verify ${{ matrix.test_args }} - - - name: Upload coverage to Codecov - if: ${{ matrix.flag }} != 'shell' - uses: codecov/codecov-action@v3 - with: - flags: ${{ matrix.flag }} + mvn -B -nsu $projects_list verify ${{ matrix.test_args }} - name: Aggregates all test reports to ./test-reports and ./surefire-reports directories if: ${{ always() }} diff --git a/.github/workflows/bk-streamstorage-python.yml b/.github/workflows/bk-streamstorage-python.yml new file mode 100644 index 00000000000..3cf7d935837 --- /dev/null +++ b/.github/workflows/bk-streamstorage-python.yml @@ -0,0 +1,85 @@ +# +# 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. +# + +name: BookKeeper StreamStorage Python Client +on: + pull_request: + branches: + - master + - branch-* + paths: + - 'stream/**' + - '.github/workflows/bk-streamstorage-python.yml' + push: + branches: + - master + - branch-* + paths: + - 'stream/**' + - '.github/workflows/bk-streamstorage-python.yml' + +jobs: + stream-storage-python-client-unit-tests: + name: StreamStorage Python Client Unit Tests + runs-on: ubuntu-latest + timeout-minutes: 60 + steps: + - name: checkout + uses: actions/checkout@v3 + - name: Tune Runner VM + uses: ./.github/actions/tune-runner-vm + - name: Test + run: ./stream/clients/python/scripts/test.sh + + + Stream-storage-python-client-integration-tests: + name: StreamStorage Python Client Integration Tests + runs-on: ubuntu-latest + timeout-minutes: 60 + steps: + - name: checkout + uses: actions/checkout@v3 + - name: Tune Runner VM + uses: ./.github/actions/tune-runner-vm + - name: Cache local Maven repository + id: cache + uses: actions/cache@v3 + with: + path: | + ~/.m2/repository/*/*/* + !~/.m2/repository/org/apache/bookkeeper + !~/.m2/repository/org/apache/distributedlog + key: ${{ runner.os }}-bookkeeper-all-${{ hashFiles('**/pom.xml') }} + - name: Set up JDK 11 + uses: actions/setup-java@v2 + with: + distribution: 'temurin' + java-version: 11 + - name: Set up Maven + uses: apache/pulsar-test-infra/setup-maven@master + with: + maven-version: 3.8.7 + - name: Build + run: mvn -q -T 1C -B -nsu clean install -DskipTests -Dcheckstyle.skip -Dspotbugs.skip -Drat.skip -Dmaven.javadoc.skip + - name: Build Test image + run: ./stream/clients/python/docker/build-local-image.sh + - name: Test + run: ./stream/clients/python/scripts/docker_integration_tests.sh + + diff --git a/bin/bkperf b/bin/bkperf index 1cd3f71fce0..02f8b20da27 100755 --- a/bin/bkperf +++ b/bin/bkperf @@ -50,6 +50,8 @@ CLI_LOG_FILE=${CLI_LOG_FILE:-"bkperf.log"} CLI_ROOT_LOG_LEVEL=${CLI_ROOT_LOG_LEVEL:-"INFO"} CLI_ROOT_LOG_APPENDER=${CLI_ROOT_LOG_APPENDER:-"CONSOLE"} +mkdir -p ${CLI_LOG_DIR} + # Configure the classpath CLI_CLASSPATH="$CLI_JAR:$CLI_CLASSPATH:$CLI_EXTRA_CLASSPATH" CLI_CLASSPATH="`dirname $CLI_LOG_CONF`:$CLI_CLASSPATH" diff --git a/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchThroughputLatency.java b/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchThroughputLatency.java index 84573eec7ca..ca2afd11c27 100644 --- a/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchThroughputLatency.java +++ b/bookkeeper-benchmark/src/main/java/org/apache/bookkeeper/benchmark/BenchThroughputLatency.java @@ -111,7 +111,9 @@ public BenchThroughputLatency(int ensemble, int writeQuorumSize, int ackQuorumSi Random rand = new Random(); public void close() throws InterruptedException, BKException { for (int i = 0; i < numberOfLedgers; i++) { - lh[i].close(); + if (lh[i] != null) { + lh[i].close(); + } } bk.close(); } @@ -257,6 +259,8 @@ public static void main(String[] args) options.addOption("skipwarmup", false, "Skip warm up, default false"); options.addOption("sendlimit", true, "Max number of entries to send. Default 20000000"); options.addOption("latencyFile", true, "File to dump latencies. Default is latencyDump.dat"); + options.addOption("useV2", false, "Whether use V2 protocol to send requests to the bookie server"); + options.addOption("warmupMessages", true, "Number of messages to warm up. Default 10000"); options.addOption("help", false, "This message"); CommandLineParser parser = new PosixParser(); @@ -281,6 +285,7 @@ public static void main(String[] args) } int throttle = Integer.parseInt(cmd.getOptionValue("throttle", "10000")); int sendLimit = Integer.parseInt(cmd.getOptionValue("sendlimit", "20000000")); + int warmupMessages = Integer.parseInt(cmd.getOptionValue("warmupMessages", "10000")); final int sockTimeout = Integer.parseInt(cmd.getOptionValue("sockettimeout", "5")); @@ -321,11 +326,15 @@ public void run() { ClientConfiguration conf = new ClientConfiguration(); conf.setThrottleValue(throttle).setReadTimeout(sockTimeout).setZkServers(servers); + if (cmd.hasOption("useV2")) { + conf.setUseV2WireProtocol(true); + } + if (!cmd.hasOption("skipwarmup")) { long throughput; LOG.info("Starting warmup"); - throughput = warmUp(data, ledgers, ensemble, quorum, passwd, conf); + throughput = warmUp(data, ledgers, ensemble, quorum, passwd, warmupMessages, conf); LOG.info("Warmup tp: " + throughput); LOG.info("Warmup phase finished"); } @@ -438,7 +447,7 @@ private static double percentile(long[] latency, int percentile) { *

TODO: update benchmark to use metadata service uri {@link https://github.com/apache/bookkeeper/issues/1331} */ private static long warmUp(byte[] data, int ledgers, int ensemble, int qSize, - byte[] passwd, ClientConfiguration conf) + byte[] passwd, int warmupMessages, ClientConfiguration conf) throws KeeperException, IOException, InterruptedException, BKException { final CountDownLatch connectLatch = new CountDownLatch(1); final int bookies; @@ -465,7 +474,7 @@ public void process(WatchedEvent event) { } BenchThroughputLatency warmup = new BenchThroughputLatency(bookies, bookies, bookies, passwd, - ledgers, 10000, conf); + ledgers, warmupMessages, conf); warmup.setEntryData(data); Thread thread = new Thread(warmup); thread.start(); diff --git a/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/LeakDetectionPolicy.java b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/LeakDetectionPolicy.java index f5c99a7e030..90d19a4aeef 100644 --- a/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/LeakDetectionPolicy.java +++ b/bookkeeper-common-allocator/src/main/java/org/apache/bookkeeper/common/allocator/LeakDetectionPolicy.java @@ -17,9 +17,12 @@ */ package org.apache.bookkeeper.common.allocator; +import lombok.extern.slf4j.Slf4j; + /** * Define the policy for the Netty leak detector. */ +@Slf4j public enum LeakDetectionPolicy { /** @@ -43,5 +46,17 @@ public enum LeakDetectionPolicy { * stack traces of places where the buffer was used. Introduce very * significant overhead. */ - Paranoid, + Paranoid; + + public static LeakDetectionPolicy parseLevel(String levelStr) { + String trimmedLevelStr = levelStr.trim(); + for (LeakDetectionPolicy policy : values()) { + if (trimmedLevelStr.equalsIgnoreCase(policy.name())) { + return policy; + } + } + log.warn("Parse leak detection policy level {} failed. Use the default level: {}", levelStr, + LeakDetectionPolicy.Disabled.name()); + return LeakDetectionPolicy.Disabled; + } } diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt index b66bc8b749f..0fce3d3d325 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt @@ -320,6 +320,7 @@ Apache Software License, Version 2. - lib/org.xerial.snappy-snappy-java-1.1.7.7.jar [50] - lib/io.reactivex.rxjava3-rxjava-3.0.1.jar [51] - lib/org.hdrhistogram-HdrHistogram-2.1.10.jar [52] +- lib/com.carrotsearch-hppc-0.9.1.jar [53] [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.13.4 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.13.4 @@ -369,6 +370,7 @@ Apache Software License, Version 2. [50] Source available at https://github.com/google/snappy/releases/tag/1.1.7.7 [51] Source available at https://github.com/ReactiveX/RxJava/tree/v3.0.1 [52] Source available at https://github.com/HdrHistogram/HdrHistogram/tree/HdrHistogram-2.1.10 +[53] Source available at https://github.com/carrotsearch/hppc/tree/0.9.1 ------------------------------------------------------------------------------------ lib/io.netty-netty-codec-4.1.89.Final.jar bundles some 3rd party dependencies diff --git a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt index 013d46207a5..e8ac8cc2c4f 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt @@ -291,6 +291,7 @@ Apache Software License, Version 2. - lib/org.conscrypt-conscrypt-openjdk-uber-2.5.1.jar [49] - lib/org.xerial.snappy-snappy-java-1.1.7.7.jar [50] - lib/io.reactivex.rxjava3-rxjava-3.0.1.jar [51] +- lib/com.carrotsearch-hppc-0.9.1.jar [52] [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.13.4 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.13.4 @@ -331,6 +332,7 @@ Apache Software License, Version 2. [49] Source available at https://github.com/google/conscrypt/releases/tag/2.5.1 [50] Source available at https://github.com/google/snappy/releases/tag/1.1.7.7 [51] Source available at https://github.com/ReactiveX/RxJava/tree/v3.0.1 +[52] Source available at https://github.com/carrotsearch/hppc/tree/0.9.1 ------------------------------------------------------------------------------------ lib/io.netty-netty-codec-4.1.89.Final.jar bundles some 3rd party dependencies diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt index cde305a40bf..51e2d3d5c81 100644 --- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt +++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt @@ -316,6 +316,7 @@ Apache Software License, Version 2. - lib/org.conscrypt-conscrypt-openjdk-uber-2.5.1.jar [49] - lib/org.xerial.snappy-snappy-java-1.1.7.7.jar [50] - lib/io.reactivex.rxjava3-rxjava-3.0.1.jar [51] +- lib/com.carrotsearch-hppc-0.9.1.jar [52] [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.13.4 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.13.4 @@ -364,6 +365,7 @@ Apache Software License, Version 2. [49] Source available at https://github.com/google/conscrypt/releases/tag/2.5.1 [50] Source available at https://github.com/google/snappy/releases/tag/1.1.7.7 [51] Source available at https://github.com/ReactiveX/RxJava/tree/v3.0.1 +[52] Source available at https://github.com/carrotsearch/hppc/tree/0.9.1 ------------------------------------------------------------------------------------ lib/io.netty-netty-codec-4.1.89.Final.jar bundles some 3rd party dependencies diff --git a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java index ddbe46b6b2f..296201e9e8b 100644 --- a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java +++ b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpRouter.java @@ -55,6 +55,7 @@ public abstract class HttpRouter { public static final String BOOKIE_IS_READY = "/api/v1/bookie/is_ready"; public static final String BOOKIE_INFO = "/api/v1/bookie/info"; public static final String CLUSTER_INFO = "/api/v1/bookie/cluster_info"; + public static final String ENTRY_LOCATION_COMPACT = "/api/v1/bookie/entry_location_compact"; // autorecovery public static final String AUTORECOVERY_STATUS = "/api/v1/autorecovery/status"; public static final String RECOVERY_BOOKIE = "/api/v1/autorecovery/bookie"; @@ -97,6 +98,8 @@ public HttpRouter(AbstractHttpHandlerFactory handlerFactory) { handlerFactory.newHandler(HttpServer.ApiType.SUSPEND_GC_COMPACTION)); this.endpointHandlers.put(RESUME_GC_COMPACTION, handlerFactory.newHandler(HttpServer.ApiType.RESUME_GC_COMPACTION)); + this.endpointHandlers.put(ENTRY_LOCATION_COMPACT, + handlerFactory.newHandler(HttpServer.ApiType.TRIGGER_ENTRY_LOCATION_COMPACT)); // autorecovery this.endpointHandlers.put(AUTORECOVERY_STATUS, handlerFactory diff --git a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java index afadba87c78..71d597d5ffa 100644 --- a/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java +++ b/bookkeeper-http/http-server/src/main/java/org/apache/bookkeeper/http/HttpServer.java @@ -36,6 +36,7 @@ enum StatusCode { BAD_REQUEST(400), FORBIDDEN(403), NOT_FOUND(404), + METHOD_NOT_ALLOWED(405), INTERNAL_ERROR(500), SERVICE_UNAVAILABLE(503); @@ -89,6 +90,7 @@ enum ApiType { CLUSTER_INFO, RESUME_GC_COMPACTION, SUSPEND_GC_COMPACTION, + TRIGGER_ENTRY_LOCATION_COMPACT, // autorecovery AUTORECOVERY_STATUS, RECOVERY_BOOKIE, diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml index 82dbadec473..a46b20c23d7 100644 --- a/bookkeeper-server/pom.xml +++ b/bookkeeper-server/pom.xml @@ -149,6 +149,10 @@ runtime true + + com.carrotsearch + hppc + org.apache.bookkeeper diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java index ac9df53cd22..90c8acf5af4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java @@ -23,7 +23,6 @@ import java.util.PrimitiveIterator; import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.common.util.Watcher; -import org.apache.bookkeeper.processor.RequestProcessor; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; /** @@ -87,8 +86,6 @@ void cancelWaitForLastAddConfirmedUpdate(long ledgerId, // TODO: Should be constructed and passed in as a parameter LedgerStorage getLedgerStorage(); - void setRequestProcessor(RequestProcessor requestProcessor); - // TODO: Move this exceptions somewhere else /** * Exception is thrown when no such a ledger is found in this bookie. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieImpl.java index 2b76488cbe9..0db230d9d3d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieImpl.java @@ -69,7 +69,6 @@ import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNS; -import org.apache.bookkeeper.processor.RequestProcessor; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; @@ -1282,11 +1281,4 @@ public OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException, NoLedg } } } - - @Override - public void setRequestProcessor(RequestProcessor requestProcessor) { - for (Journal journal : journals) { - journal.setRequestProcessor(requestProcessor); - } - } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileSystemUpgrade.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileSystemUpgrade.java index bb6cd2404a3..8fff510c562 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileSystemUpgrade.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileSystemUpgrade.java @@ -24,6 +24,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithRegistrationManager; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import com.google.common.util.concurrent.UncheckedExecutionException; import java.io.File; @@ -31,6 +32,7 @@ import java.io.IOException; import java.net.MalformedURLException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -96,10 +98,17 @@ public boolean accept(File dir, String name) { } }; - private static List getAllDirectories(ServerConfiguration conf) { + @VisibleForTesting + public static List getAllDirectories(ServerConfiguration conf) { List dirs = new ArrayList<>(); dirs.addAll(Lists.newArrayList(conf.getJournalDirs())); - Collections.addAll(dirs, conf.getLedgerDirs()); + final File[] ledgerDirs = conf.getLedgerDirs(); + final File[] indexDirs = conf.getIndexDirs(); + if (indexDirs != null && indexDirs.length == ledgerDirs.length + && !Arrays.asList(indexDirs).containsAll(Arrays.asList(ledgerDirs))) { + dirs.addAll(Lists.newArrayList(indexDirs)); + } + Collections.addAll(dirs, ledgerDirs); return dirs; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java index d17450426e7..5de2292263a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java @@ -218,7 +218,7 @@ public void removeEntryLog(long logToRemove) { this.throttler = new AbstractLogCompactor.Throttler(conf); if (minorCompactionInterval > 0 && minorCompactionThreshold > 0) { - if (minorCompactionThreshold > 1.0f) { + if (minorCompactionThreshold > 1.0d) { throw new IOException("Invalid minor compaction threshold " + minorCompactionThreshold); } @@ -230,16 +230,16 @@ public void removeEntryLog(long logToRemove) { } if (isForceAllowCompaction) { - if (minorCompactionThreshold > 0 && minorCompactionThreshold < 1.0f) { + if (minorCompactionThreshold > 0 && minorCompactionThreshold < 1.0d) { isForceMinorCompactionAllow = true; } - if (majorCompactionThreshold > 0 && majorCompactionThreshold < 1.0f) { + if (majorCompactionThreshold > 0 && majorCompactionThreshold < 1.0d) { isForceMajorCompactionAllow = true; } } if (majorCompactionInterval > 0 && majorCompactionThreshold > 0) { - if (majorCompactionThreshold > 1.0f) { + if (majorCompactionThreshold > 1.0d) { throw new IOException("Invalid major compaction threshold " + majorCompactionThreshold); } @@ -410,13 +410,14 @@ public void runWithFlags(boolean force, boolean suspendMajor, boolean suspendMin compactor.cleanUpAndRecover(); try { - // Extract all of the ledger ID's that comprise all of the entry logs - // (except for the current new one which is still being written to). - extractMetaFromEntryLogs(); - // gc inactive/deleted ledgers + // this is used in extractMetaFromEntryLogs to calculate the usage of entry log doGcLedgers(); + // Extract all of the ledger ID's that comprise all of the entry logs + // (except for the current new one which is still being written to). + extractMetaFromEntryLogs(); + // gc entry logs doGcEntryLogs(); @@ -561,15 +562,20 @@ void doCompactEntryLogs(double threshold, long maxTimeMillis) throws EntryLogMet MutableLong timeDiff = new MutableLong(0); entryLogMetaMap.forEach((entryLogId, meta) -> { - int bucketIndex = calculateUsageIndex(numBuckets, meta.getUsage()); + double usage = meta.getUsage(); + if (conf.isUseTargetEntryLogSizeForGc() && usage < 1.0d) { + usage = (double) meta.getRemainingSize() / Math.max(meta.getTotalSize(), conf.getEntryLogSizeLimit()); + } + int bucketIndex = calculateUsageIndex(numBuckets, usage); entryLogUsageBuckets[bucketIndex]++; if (timeDiff.getValue() < maxTimeMillis) { end.setValue(System.currentTimeMillis()); timeDiff.setValue(end.getValue() - start); } - if (meta.getUsage() >= threshold || (maxTimeMillis > 0 && timeDiff.getValue() >= maxTimeMillis) - || !running) { + if ((usage >= threshold + || (maxTimeMillis > 0 && timeDiff.getValue() >= maxTimeMillis) + || !running)) { // We allow the usage limit calculation to continue so that we get an accurate // report of where the usage was prior to running compaction. return; @@ -745,6 +751,7 @@ protected void extractMetaFromEntryLogs() throws EntryLogMetadataMapException { EntryLogMetadata entryLogMeta = entryLogger.getEntryLogMetadata(entryLogId, throttler); removeIfLedgerNotExists(entryLogMeta); if (entryLogMeta.isEmpty()) { + LOG.info("Entry log file {} is empty, delete it from disk.", Long.toHexString(entryLogId)); entryLogger.removeEntryLog(entryLogId); // remove it from entrylogmetadata-map if it is present in // the map diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java index cdf68cd85d1..48df4e9b1e4 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Journal.java @@ -21,6 +21,8 @@ package org.apache.bookkeeper.bookie; +import com.carrotsearch.hppc.ObjectHashSet; +import com.carrotsearch.hppc.procedures.ObjectProcedure; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Stopwatch; import io.netty.buffer.ByteBuf; @@ -37,7 +39,6 @@ import java.nio.ByteBuffer; import java.nio.channels.FileChannel; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.concurrent.ThreadFactory; @@ -51,7 +52,7 @@ import org.apache.bookkeeper.common.util.MemoryLimitController; import org.apache.bookkeeper.common.util.affinity.CpuAffinity; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.processor.RequestProcessor; +import org.apache.bookkeeper.proto.BookieRequestHandler; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.NullStatsLogger; @@ -333,6 +334,10 @@ public void run() { recycle(); } + private Object getCtx() { + return ctx; + } + private final Handle recyclerHandle; private QueueEntry(Handle recyclerHandle) { @@ -366,14 +371,19 @@ public static class ForceWriteRequest { private boolean shouldClose; private long lastFlushedPosition; private long logId; + private boolean flushed; - public int process() { + public int process(ObjectHashSet writeHandlers) { closeFileIfNecessary(); // Notify the waiters that the force write succeeded for (int i = 0; i < forceWriteWaiters.size(); i++) { QueueEntry qe = forceWriteWaiters.get(i); if (qe != null) { + if (qe.getCtx() instanceof BookieRequestHandler + && qe.entryId != BookieImpl.METAENTRY_ID_FORCE_LEDGER) { + writeHandlers.add((BookieRequestHandler) qe.getCtx()); + } qe.run(); } } @@ -381,13 +391,20 @@ public int process() { return forceWriteWaiters.size(); } + private void flushFileToDisk() throws IOException { + if (!flushed) { + logFile.forceWrite(false); + flushed = true; + } + } + public void closeFileIfNecessary() { // Close if shouldClose is set if (shouldClose) { // We should guard against exceptions so its // safe to call in catch blocks try { - logFile.forceWrite(false); + flushFileToDisk(); logFile.close(); // Call close only once shouldClose = false; @@ -446,7 +463,6 @@ private class ForceWriteThread extends BookieCriticalThread { // successful force write Thread threadToNotifyOnEx; - RequestProcessor requestProcessor; // should we group force writes private final boolean enableGroupForceWrites; private final Counter forceWriteThreadTime; @@ -473,6 +489,7 @@ public void run() { } final ForceWriteRequest[] localRequests = new ForceWriteRequest[conf.getJournalQueueSize()]; + final ObjectHashSet writeHandlers = new ObjectHashSet<>(); while (running) { try { @@ -490,17 +507,16 @@ public void run() { // responses for (int i = 0; i < requestsCount; i++) { ForceWriteRequest req = localRequests[i]; - numEntriesInLastForceWrite += req.process(); + numEntriesInLastForceWrite += req.process(writeHandlers); + localRequests[i] = null; req.recycle(); } - Arrays.fill(localRequests, 0, requestsCount, null); journalStats.getForceWriteGroupingCountStats() .registerSuccessfulValue(numEntriesInLastForceWrite); - - if (requestProcessor != null) { - requestProcessor.flushPendingResponses(); - } - + writeHandlers.forEach( + (ObjectProcedure) + BookieRequestHandler::flushPendingResponse); + writeHandlers.clear(); } catch (IOException ioe) { LOG.error("I/O exception in ForceWrite thread", ioe); running = false; @@ -519,7 +535,7 @@ public void run() { private void syncJournal(ForceWriteRequest lastRequest) throws IOException { long fsyncStartTime = MathUtils.nowInNano(); try { - lastRequest.logFile.forceWrite(false); + lastRequest.flushFileToDisk(); journalStats.getJournalSyncStats().registerSuccessfulEvent(MathUtils.elapsedNanos(fsyncStartTime), TimeUnit.NANOSECONDS); lastLogMark.setCurLogMark(lastRequest.logId, lastRequest.lastFlushedPosition); @@ -884,7 +900,7 @@ public void logAddEntry(long ledgerId, long entryId, ByteBuf entry, memoryLimitController.reserveMemory(entry.readableBytes()); queue.put(QueueEntry.create( - entry, ackBeforeSync, ledgerId, entryId, cb, ctx, MathUtils.nowInNano(), + entry, ackBeforeSync, ledgerId, entryId, cb, ctx, MathUtils.nowInNano(), journalStats.getJournalAddEntryStats(), callbackTime)); } @@ -964,6 +980,9 @@ public void run() { int localQueueEntriesIdx = 0; int localQueueEntriesLen = 0; + final ObjectHashSet writeHandlers = new ObjectHashSet<>(); + + QueueEntry qe = null; while (true) { // new journal file to write @@ -1020,7 +1039,8 @@ journalFormatVersionToWrite, getBufferedChannelBuilder(), dequeueStartTime = MathUtils.nowInNano(); if (localQueueEntriesLen > 0) { - qe = localQueueEntries[localQueueEntriesIdx++]; + qe = localQueueEntries[localQueueEntriesIdx]; + localQueueEntries[localQueueEntriesIdx++] = null; journalStats.getJournalQueueSize().dec(); journalStats.getJournalQueueStats() .registerSuccessfulEvent(MathUtils.elapsedNanos(qe.enqueueTime), TimeUnit.NANOSECONDS); @@ -1080,12 +1100,17 @@ journalFormatVersionToWrite, getBufferedChannelBuilder(), if (entry != null && (!syncData || entry.ackBeforeSync)) { toFlush.set(i, null); numEntriesToFlush--; + if (entry.getCtx() instanceof BookieRequestHandler + && entry.entryId != BookieImpl.METAENTRY_ID_FORCE_LEDGER) { + writeHandlers.add((BookieRequestHandler) entry.getCtx()); + } entry.run(); } - if (forceWriteThread.requestProcessor != null) { - forceWriteThread.requestProcessor.flushPendingResponses(); - } } + writeHandlers.forEach( + (ObjectProcedure) + BookieRequestHandler::flushPendingResponse); + writeHandlers.clear(); lastFlushPosition = bc.position(); journalStats.getJournalFlushStats().registerSuccessfulEvent( @@ -1179,9 +1204,9 @@ journalFormatVersionToWrite, getBufferedChannelBuilder(), numEntriesToFlush++; if (localQueueEntriesIdx < localQueueEntriesLen) { - qe = localQueueEntries[localQueueEntriesIdx++]; + qe = localQueueEntries[localQueueEntriesIdx]; + localQueueEntries[localQueueEntriesIdx++] = null; } else { - Arrays.fill(localQueueEntries, 0, localQueueEntriesLen, null); qe = null; } } @@ -1208,10 +1233,6 @@ public BufferedChannelBuilder getBufferedChannelBuilder() { return (FileChannel fc, int capacity) -> new BufferedChannel(allocator, fc, capacity); } - public void setRequestProcessor(RequestProcessor requestProcessor) { - forceWriteThread.requestProcessor = requestProcessor; - } - /** * Shuts down the journal. */ diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java index bedc7d26814..92f0d38c6ea 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java @@ -29,6 +29,7 @@ import java.util.Collections; import java.util.EnumSet; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.PrimitiveIterator; import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; @@ -259,6 +260,26 @@ default boolean isMinorGcSuspended() { return false; } + default void entryLocationCompact() { + return; + } + + default void entryLocationCompact(List locations) { + return; + } + + default boolean isEntryLocationCompacting() { + return false; + } + + default Map isEntryLocationCompacting(List locations) { + return Collections.emptyMap(); + } + + default List getEntryLocationDBPath() { + return Collections.emptyList(); + } + /** * Class for describing location of a generic inconsistency. Implementations should * ensure that detail is populated with an exception which adequately describes the diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java index e02615df63d..ce9e8370ea5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java @@ -26,6 +26,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import io.netty.util.concurrent.DefaultThreadFactory; @@ -34,7 +35,9 @@ import java.io.IOException; import java.util.ArrayList; import java.util.EnumSet; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.PrimitiveIterator.OfLong; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -542,6 +545,47 @@ public boolean isMinorGcSuspended() { return ledgerStorageList.stream().allMatch(SingleDirectoryDbLedgerStorage::isMinorGcSuspended); } + @Override + public void entryLocationCompact() { + ledgerStorageList.forEach(SingleDirectoryDbLedgerStorage::entryLocationCompact); + } + + @Override + public void entryLocationCompact(List locations) { + for (SingleDirectoryDbLedgerStorage ledgerStorage : ledgerStorageList) { + String entryLocation = ledgerStorage.getEntryLocationDBPath().get(0); + if (locations.contains(entryLocation)) { + ledgerStorage.entryLocationCompact(); + } + } + } + + @Override + public boolean isEntryLocationCompacting() { + return ledgerStorageList.stream().anyMatch(SingleDirectoryDbLedgerStorage::isEntryLocationCompacting); + } + + @Override + public Map isEntryLocationCompacting(List locations) { + HashMap isCompacting = Maps.newHashMap(); + for (SingleDirectoryDbLedgerStorage ledgerStorage : ledgerStorageList) { + String entryLocation = ledgerStorage.getEntryLocationDBPath().get(0); + if (locations.contains(entryLocation)) { + isCompacting.put(entryLocation, ledgerStorage.isEntryLocationCompacting()); + } + } + return isCompacting; + } + + @Override + public List getEntryLocationDBPath() { + List allEntryLocationDBPath = Lists.newArrayList(); + for (SingleDirectoryDbLedgerStorage ledgerStorage : ledgerStorageList) { + allEntryLocationDBPath.addAll(ledgerStorage.getEntryLocationDBPath()); + } + return allEntryLocationDBPath; + } + @Override public List getGarbageCollectionStatus() { return ledgerStorageList.stream() diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java index 3f6d1ae55b2..335609aa860 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java @@ -48,6 +48,7 @@ public class EntryLocationIndex implements Closeable { private final KeyValueStorage locationsDb; private final ConcurrentLongHashSet deletedLedgers = ConcurrentLongHashSet.newBuilder().build(); private final EntryLocationIndexStats stats; + private boolean isCompacting; public EntryLocationIndex(ServerConfiguration conf, KeyValueStorageFactory storageFactory, String basePath, StatsLogger stats) throws IOException { @@ -189,6 +190,23 @@ public void delete(long ledgerId) throws IOException { deletedLedgers.add(ledgerId); } + public String getEntryLocationDBPath() { + return locationsDb.getDBPath(); + } + + public void compact() throws IOException { + try { + isCompacting = true; + locationsDb.compact(); + } finally { + isCompacting = false; + } + } + + public boolean isCompacting() { + return isCompacting; + } + public void removeOffsetFromDeletedLedgers() throws IOException { LongPairWrapper firstKeyWrapper = LongPairWrapper.get(-1, -1); LongPairWrapper lastKeyWrapper = LongPairWrapper.get(-1, -1); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorage.java index 08e7ad7396f..ab724d73bb5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorage.java @@ -106,6 +106,16 @@ public interface KeyValueStorage extends Closeable { */ default void compact(byte[] firstKey, byte[] lastKey) throws IOException {} + /** + * Compact storage full range. + */ + default void compact() throws IOException {} + + /** + * Get storage path. + */ + String getDBPath(); + /** * Get an iterator over to scan sequentially through all the keys in the * database. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java index d402e66e118..fd870506939 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java @@ -51,6 +51,7 @@ import org.rocksdb.Env; import org.rocksdb.InfoLogLevel; import org.rocksdb.LRUCache; +import org.rocksdb.LiveFileMetaData; import org.rocksdb.Options; import org.rocksdb.OptionsUtil; import org.rocksdb.ReadOptions; @@ -84,6 +85,8 @@ public class KeyValueStorageRocksDB implements KeyValueStorage { private final ReadOptions optionDontCache; private final WriteBatch emptyBatch; + private String dbPath; + private static final String ROCKSDB_LOG_PATH = "dbStorage_rocksDB_logPath"; private static final String ROCKSDB_LOG_LEVEL = "dbStorage_rocksDB_logLevel"; private static final String ROCKSDB_LZ4_COMPRESSION_ENABLED = "dbStorage_rocksDB_lz4CompressionEnabled"; @@ -158,13 +161,13 @@ private RocksDB initializeRocksDBWithConfFile(String basePath, String subPath, D log.info("RocksDB<{}> log path: {}", subPath, logPathSetting); dbOptions.setDbLogDir(logPathSetting.toString()); } - String path = FileSystems.getDefault().getPath(basePath, subPath).toFile().toString(); + this.dbPath = FileSystems.getDefault().getPath(basePath, subPath).toFile().toString(); this.options = dbOptions; this.columnFamilyDescriptors = cfDescs; if (readOnly) { - return RocksDB.openReadOnly(dbOptions, path, cfDescs, cfHandles); + return RocksDB.openReadOnly(dbOptions, dbPath, cfDescs, cfHandles); } else { - return RocksDB.open(dbOptions, path, cfDescs, cfHandles); + return RocksDB.open(dbOptions, dbPath, cfDescs, cfHandles); } } catch (RocksDBException e) { throw new IOException("Error open RocksDB database", e); @@ -241,7 +244,7 @@ private RocksDB initializeRocksDBWithBookieConf(String basePath, String subPath, log.info("RocksDB<{}> log path: {}", subPath, logPathSetting); options.setDbLogDir(logPathSetting.toString()); } - String path = FileSystems.getDefault().getPath(basePath, subPath).toFile().toString(); + this.dbPath = FileSystems.getDefault().getPath(basePath, subPath).toFile().toString(); // Configure log level String logLevel = conf.getString(ROCKSDB_LOG_LEVEL, "info"); @@ -268,9 +271,9 @@ private RocksDB initializeRocksDBWithBookieConf(String basePath, String subPath, this.options = options; try { if (readOnly) { - return RocksDB.openReadOnly(options, path); + return RocksDB.openReadOnly(options, dbPath); } else { - return RocksDB.open(options, path); + return RocksDB.open(options, dbPath); } } catch (RocksDBException e) { throw new IOException("Error open RocksDB database", e); @@ -365,6 +368,11 @@ public void delete(byte[] key) throws IOException { } } + @Override + public String getDBPath() { + return dbPath; + } + @Override public void compact(byte[] firstKey, byte[] lastKey) throws IOException { try { @@ -374,6 +382,36 @@ public void compact(byte[] firstKey, byte[] lastKey) throws IOException { } } + @Override + public void compact() throws IOException { + try { + final long start = System.currentTimeMillis(); + final int oriRocksDBFileCount = db.getLiveFilesMetaData().size(); + final long oriRocksDBSize = getRocksDBSize(); + log.info("Starting RocksDB {} compact, current RocksDB hold {} files and {} Bytes.", + db.getName(), oriRocksDBFileCount, oriRocksDBSize); + + db.compactRange(); + + final long end = System.currentTimeMillis(); + final int rocksDBFileCount = db.getLiveFilesMetaData().size(); + final long rocksDBSize = getRocksDBSize(); + log.info("RocksDB {} compact finished {} ms, space reduced {} Bytes, current hold {} files and {} Bytes.", + db.getName(), end - start, oriRocksDBSize - rocksDBSize, rocksDBFileCount, rocksDBSize); + } catch (RocksDBException e) { + throw new IOException("Error in RocksDB compact", e); + } + } + + private long getRocksDBSize() { + List liveFilesMetaData = db.getLiveFilesMetaData(); + long rocksDBFileSize = 0L; + for (LiveFileMetaData fileMetaData : liveFilesMetaData) { + rocksDBFileSize += fileMetaData.size(); + } + return rocksDBFileSize; + } + @Override public void sync() throws IOException { try { 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 565732d4284..635a2727f9d 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 @@ -298,6 +298,34 @@ public boolean isMinorGcSuspended() { return gcThread.isMinorGcSuspend(); } + @Override + public void entryLocationCompact() { + if (entryLocationIndex.isCompacting()) { + // RocksDB already running compact. + return; + } + cleanupExecutor.execute(() -> { + // There can only be one single cleanup task running because the cleanupExecutor + // is single-threaded + try { + log.info("Trigger entry location index RocksDB compact."); + entryLocationIndex.compact(); + } catch (Throwable t) { + log.warn("Failed to trigger entry location index RocksDB compact", t); + } + }); + } + + @Override + public boolean isEntryLocationCompacting() { + return entryLocationIndex.isCompacting(); + } + + @Override + public List getEntryLocationDBPath() { + return Lists.newArrayList(entryLocationIndex.getEntryLocationDBPath()); + } + @Override public void shutdown() throws InterruptedException { try { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragmentReplicator.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragmentReplicator.java index 877a3ac300a..6b439b0960d 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragmentReplicator.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragmentReplicator.java @@ -29,7 +29,9 @@ import static org.apache.bookkeeper.replication.ReplicationStats.WRITE_DATA_LATENCY; import com.google.common.util.concurrent.RateLimiter; +import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import io.netty.util.ReferenceCounted; import java.util.Enumeration; import java.util.HashSet; import java.util.Iterator; @@ -403,17 +405,24 @@ public void readComplete(int rc, LedgerHandle lh, numEntriesRead.inc(); numBytesRead.registerSuccessfulValue(dataLength); - ByteBufList toSend = lh.getDigestManager() + ReferenceCounted toSend = lh.getDigestManager() .computeDigestAndPackageForSending(entryId, lh.getLastAddConfirmed(), entry.getLength(), - Unpooled.wrappedBuffer(data, 0, data.length)); + Unpooled.wrappedBuffer(data, 0, data.length), + lh.getLedgerKey(), + 0 + ); if (replicationThrottle != null) { - updateAverageEntrySize(toSend.readableBytes()); + if (toSend instanceof ByteBuf) { + updateAverageEntrySize(((ByteBuf) toSend).readableBytes()); + } else if (toSend instanceof ByteBufList) { + updateAverageEntrySize(((ByteBufList) toSend).readableBytes()); + } } for (BookieId newBookie : newBookies) { long startWriteEntryTime = MathUtils.nowInNano(); bkc.getBookieClient().addEntry(newBookie, lh.getId(), - lh.getLedgerKey(), entryId, ByteBufList.clone(toSend), + lh.getLedgerKey(), entryId, toSend, multiWriteCallback, dataLength, BookieProtocol.FLAG_RECOVERY_ADD, false, WriteFlag.NONE); writeDataLatency.registerSuccessfulEvent( diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java index 394c961cbc9..51f559a86cc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java @@ -28,6 +28,7 @@ import io.netty.util.Recycler; import io.netty.util.Recycler.Handle; import io.netty.util.ReferenceCountUtil; +import io.netty.util.ReferenceCounted; import java.util.EnumSet; import java.util.HashSet; import java.util.List; @@ -38,7 +39,6 @@ import org.apache.bookkeeper.client.api.WriteFlag; import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; -import org.apache.bookkeeper.util.ByteBufList; import org.apache.bookkeeper.util.MathUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,7 +56,7 @@ class PendingAddOp implements WriteCallback { private static final Logger LOG = LoggerFactory.getLogger(PendingAddOp.class); ByteBuf payload; - ByteBufList toSend; + ReferenceCounted toSend; AddCallbackWithLatency cb; Object ctx; long entryId; @@ -242,9 +242,10 @@ public synchronized void initiate() { checkNotNull(lh); checkNotNull(lh.macManager); + int flags = isRecoveryAdd ? FLAG_RECOVERY_ADD | FLAG_HIGH_PRIORITY : FLAG_NONE; this.toSend = lh.macManager.computeDigestAndPackageForSending( entryId, lh.lastAddConfirmed, currentLedgerLength, - payload); + payload, lh.ledgerKey, flags); // ownership of RefCounted ByteBuf was passed to computeDigestAndPackageForSending payload = null; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java index 21bde894a36..438dc40983e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/AbstractConfiguration.java @@ -1129,8 +1129,17 @@ public T setAllocatorOutOfMemoryPolicy(OutOfMemoryPolicy oomPolicy) { * Return the configured leak detection policy for the allocator. */ public LeakDetectionPolicy getAllocatorLeakDetectionPolicy() { - return LeakDetectionPolicy - .valueOf(this.getString(ALLOCATOR_LEAK_DETECTION_POLICY, LeakDetectionPolicy.Disabled.toString())); + //see: https://lists.apache.org/thread/d3zw8bxhlg0wxfhocyjglq0nbxrww3sg + String nettyLevelStr = System.getProperty("io.netty.leakDetectionLevel", LeakDetectionPolicy.Disabled.name()); + nettyLevelStr = System.getProperty("io.netty.leakDetection.level", nettyLevelStr); + String bkLevelStr = getString(ALLOCATOR_LEAK_DETECTION_POLICY, LeakDetectionPolicy.Disabled.name()); + LeakDetectionPolicy nettyLevel = LeakDetectionPolicy.parseLevel(nettyLevelStr); + LeakDetectionPolicy bkLevel = LeakDetectionPolicy.parseLevel(bkLevelStr); + if (nettyLevel.ordinal() >= bkLevel.ordinal()) { + return nettyLevel; + } else { + return bkLevel; + } } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java index 3fb8d18ef9c..cf74f6af01c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java @@ -116,6 +116,7 @@ public class ServerConfiguration extends AbstractConfigurationr */ void processRequest(Object r, BookieRequestHandler channel); - - /** - * Flush any pending response staged on all the client connections. - */ - void flushPendingResponses(); } 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 ba5dc9948dc..f923b61ad50 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 @@ -23,6 +23,7 @@ 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; import io.netty.channel.ChannelHandlerContext; @@ -39,6 +40,7 @@ import org.apache.bookkeeper.auth.ClientAuthProvider; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.proto.BookkeeperProtocol.AuthMessage; +import org.apache.bookkeeper.util.ByteBufList; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -358,8 +360,10 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) } else { waitingForAuth.add(msg); } + } else if (msg instanceof ByteBuf || msg instanceof ByteBufList) { + waitingForAuth.add(msg); } else { - LOG.info("dropping write of message {}", msg); + LOG.info("[{}] dropping write of message {}", ctx.channel(), msg); } } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClient.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClient.java index 81be386f7ca..938874fac04 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClient.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClient.java @@ -20,6 +20,7 @@ */ package org.apache.bookkeeper.proto; +import io.netty.util.ReferenceCounted; import java.util.EnumSet; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -139,7 +140,7 @@ void writeLac(BookieId address, long ledgerId, byte[] masterKey, * {@link org.apache.bookkeeper.client.api.WriteFlag} */ void addEntry(BookieId address, long ledgerId, byte[] masterKey, - long entryId, ByteBufList toSend, WriteCallback cb, Object ctx, + long entryId, ReferenceCounted toSend, WriteCallback cb, Object ctx, int options, boolean allowFastFail, EnumSet writeFlags); /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java index cd11bc17d7a..c305a51ea42 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClientImpl.java @@ -32,6 +32,7 @@ import io.netty.util.Recycler; import io.netty.util.Recycler.Handle; import io.netty.util.ReferenceCountUtil; +import io.netty.util.ReferenceCounted; import io.netty.util.concurrent.DefaultThreadFactory; import java.io.IOException; import java.util.EnumSet; @@ -288,7 +289,7 @@ public void addEntry(final BookieId addr, final long ledgerId, final byte[] masterKey, final long entryId, - final ByteBufList toSend, + final ReferenceCounted toSend, final WriteCallback cb, final Object ctx, final int options, @@ -357,7 +358,7 @@ private static class ChannelReadyForAddEntryCallback private final Handle recyclerHandle; private BookieClientImpl bookieClient; - private ByteBufList toSend; + private ReferenceCounted toSend; private long ledgerId; private long entryId; private BookieId addr; @@ -369,7 +370,7 @@ private static class ChannelReadyForAddEntryCallback private EnumSet writeFlags; static ChannelReadyForAddEntryCallback create( - BookieClientImpl bookieClient, ByteBufList toSend, long ledgerId, + BookieClientImpl bookieClient, ReferenceCounted toSend, long ledgerId, long entryId, BookieId addr, Object ctx, WriteCallback cb, int options, byte[] masterKey, boolean allowFastFail, EnumSet writeFlags) { 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 edbffa5f431..c56235dbe67 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 @@ -110,23 +110,7 @@ public Object encode(Object msg, ByteBufAllocator allocator) return msg; } BookieProtocol.Request r = (BookieProtocol.Request) msg; - if (r instanceof BookieProtocol.AddRequest) { - BookieProtocol.AddRequest ar = (BookieProtocol.AddRequest) r; - ByteBufList data = ar.getData(); - - int totalHeaderSize = 4 // for the request header - + BookieProtocol.MASTER_KEY_LENGTH; // for the master key - - int totalPayloadSize = totalHeaderSize + data.readableBytes(); - ByteBuf buf = allocator.buffer(totalHeaderSize + 4 /* frame size */); - buf.writeInt(totalPayloadSize); // Frame header - buf.writeInt(PacketHeader.toInt(r.getProtocolVersion(), r.getOpCode(), r.getFlags())); - buf.writeBytes(r.getMasterKey(), 0, BookieProtocol.MASTER_KEY_LENGTH); - - ar.recycle(); - data.prepend(buf); - return data; - } else if (r instanceof BookieProtocol.ReadRequest) { + if (r instanceof BookieProtocol.ReadRequest) { int totalHeaderSize = 4 // for request type + 8 // for ledgerId + 8; // for entryId @@ -142,7 +126,7 @@ public Object encode(Object msg, ByteBufAllocator allocator) if (r.hasMasterKey()) { buf.writeBytes(r.getMasterKey(), 0, BookieProtocol.MASTER_KEY_LENGTH); } - + r.recycle(); return buf; } else if (r instanceof BookieProtocol.AuthRequest) { BookkeeperProtocol.AuthMessage am = ((BookieProtocol.AuthRequest) r).getAuthMessage(); @@ -193,9 +177,9 @@ public Object decode(ByteBuf packet) if ((flags & BookieProtocol.FLAG_DO_FENCING) == BookieProtocol.FLAG_DO_FENCING && version >= 2) { byte[] masterKey = readMasterKey(packet); - return new BookieProtocol.ReadRequest(version, ledgerId, entryId, flags, masterKey); + return BookieProtocol.ReadRequest.create(version, ledgerId, entryId, flags, masterKey); } else { - return new BookieProtocol.ReadRequest(version, ledgerId, entryId, flags, null); + return BookieProtocol.ReadRequest.create(version, ledgerId, entryId, flags, null); } case BookieProtocol.AUTH: BookkeeperProtocol.AuthMessage.Builder builder = BookkeeperProtocol.AuthMessage.newBuilder(); @@ -271,6 +255,7 @@ public Object encode(Object msg, ByteBufAllocator allocator) if (isSmallEntry) { buf.writeBytes(rr.getData()); + rr.release(); return buf; } else { return ByteBufList.get(buf, rr.getData()); @@ -436,7 +421,9 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) if (LOG.isTraceEnabled()) { LOG.trace("Encode request {} to channel {}.", msg, ctx.channel()); } - if (msg instanceof BookkeeperProtocol.Request) { + if (msg instanceof ByteBuf || msg instanceof ByteBufList) { + ctx.write(msg, promise); + } else if (msg instanceof BookkeeperProtocol.Request) { ctx.write(reqV3.encode(msg, ctx.alloc()), promise); } else if (msg instanceof BookieProtocol.Request) { ctx.write(reqPreV3.encode(msg, ctx.alloc()), promise); 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 89654449aad..3a27f08a95d 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 @@ -27,7 +27,6 @@ import io.netty.util.ReferenceCountUtil; import io.netty.util.ReferenceCounted; import org.apache.bookkeeper.proto.BookkeeperProtocol.AuthMessage; -import org.apache.bookkeeper.util.ByteBufList; /** * The packets of the Bookie protocol all have a 4-byte integer indicating the @@ -252,58 +251,6 @@ public String toString() { public void recycle() {} } - /** - * A Request that adds data. - */ - class AddRequest extends Request { - ByteBufList data; - - static AddRequest create(byte protocolVersion, long ledgerId, - long entryId, short flags, byte[] masterKey, - ByteBufList data) { - AddRequest add = RECYCLER.get(); - add.protocolVersion = protocolVersion; - add.opCode = ADDENTRY; - add.ledgerId = ledgerId; - add.entryId = entryId; - add.flags = flags; - add.masterKey = masterKey; - add.data = data.retain(); - return add; - } - - ByteBufList getData() { - // We need to have different ByteBufList instances for each bookie write - return ByteBufList.clone(data); - } - - boolean isRecoveryAdd() { - return (flags & FLAG_RECOVERY_ADD) == FLAG_RECOVERY_ADD; - } - - private final Handle recyclerHandle; - private AddRequest(Handle recyclerHandle) { - this.recyclerHandle = recyclerHandle; - } - - private static final Recycler RECYCLER = new Recycler() { - @Override - protected AddRequest newObject(Handle handle) { - return new AddRequest(handle); - } - }; - - @Override - public void recycle() { - ledgerId = -1; - entryId = -1; - masterKey = null; - ReferenceCountUtil.release(data); - data = null; - recyclerHandle.recycle(this); - } - } - /** * This is similar to add request, but it used when processing the request on the bookie side. */ @@ -362,14 +309,43 @@ public void recycle() { * A Request that reads data. */ class ReadRequest extends Request { - ReadRequest(byte protocolVersion, long ledgerId, long entryId, - short flags, byte[] masterKey) { - init(protocolVersion, READENTRY, ledgerId, entryId, flags, masterKey); + + static ReadRequest create(byte protocolVersion, long ledgerId, long entryId, + short flags, byte[] masterKey) { + ReadRequest read = RECYCLER.get(); + read.protocolVersion = protocolVersion; + read.opCode = READENTRY; + read.ledgerId = ledgerId; + read.entryId = entryId; + read.flags = flags; + read.masterKey = masterKey; + return read; } boolean isFencing() { return (flags & FLAG_DO_FENCING) == FLAG_DO_FENCING; } + + private final Handle recyclerHandle; + + private ReadRequest(Handle recyclerHandle) { + this.recyclerHandle = recyclerHandle; + } + + private static final Recycler RECYCLER = new Recycler() { + @Override + protected ReadRequest newObject(Handle handle) { + return new ReadRequest(handle); + } + }; + + @Override + public void recycle() { + ledgerId = -1; + entryId = -1; + masterKey = null; + recyclerHandle.recycle(this); + } } /** 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 50b7969023e..3d906dba449 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 @@ -35,7 +35,7 @@ @Slf4j public class BookieRequestHandler extends ChannelInboundHandlerAdapter { - static final Object EVENT_FLUSH_ALL_PENDING_RESPONSES = new Object(); + private static final int DEFAULT_PENDING_RESPONSE_SIZE = 256; private final RequestProcessor requestProcessor; private final ChannelGroup allChannels; @@ -43,7 +43,7 @@ public class BookieRequestHandler extends ChannelInboundHandlerAdapter { private ChannelHandlerContext ctx; private ByteBuf pendingSendResponses = null; - private int maxPendingResponsesSize; + private int maxPendingResponsesSize = DEFAULT_PENDING_RESPONSE_SIZE; BookieRequestHandler(ServerConfiguration conf, RequestProcessor processor, ChannelGroup allChannels) { this.requestProcessor = processor; @@ -56,7 +56,7 @@ public ChannelHandlerContext ctx() { @Override public void channelActive(ChannelHandlerContext ctx) throws Exception { - log.info("Channel connected {}", ctx.channel()); + log.info("Channel connected {}", ctx.channel()); this.ctx = ctx; super.channelActive(ctx); } @@ -92,31 +92,22 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception public synchronized void prepareSendResponseV2(int rc, BookieProtocol.ParsedAddRequest req) { if (pendingSendResponses == null) { - pendingSendResponses = ctx.alloc().directBuffer(maxPendingResponsesSize != 0 - ? maxPendingResponsesSize : 256); + pendingSendResponses = ctx().alloc().directBuffer(maxPendingResponsesSize); } - BookieProtoEncoding.ResponseEnDeCoderPreV3.serializeAddResponseInto(rc, req, pendingSendResponses); } - @Override - public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { - if (evt == EVENT_FLUSH_ALL_PENDING_RESPONSES) { - synchronized (this) { - if (pendingSendResponses != null) { - maxPendingResponsesSize = Math.max(maxPendingResponsesSize, - pendingSendResponses.readableBytes()); - if (ctx.channel().isActive()) { - ctx.writeAndFlush(pendingSendResponses, ctx.voidPromise()); - } else { - pendingSendResponses.release(); - } - - pendingSendResponses = null; - } + public synchronized void flushPendingResponse() { + if (pendingSendResponses != null) { + maxPendingResponsesSize = (int) Math.max( + maxPendingResponsesSize * 0.5 + 0.5 * pendingSendResponses.readableBytes(), + DEFAULT_PENDING_RESPONSE_SIZE); + if (ctx().channel().isActive()) { + ctx().writeAndFlush(pendingSendResponses, ctx.voidPromise()); + } else { + pendingSendResponses.release(); } - } else { - super.userEventTriggered(ctx, evt); + pendingSendResponses = null; } } } 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 d07aa9cffa0..6e7f5abcbb7 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 @@ -699,13 +699,6 @@ private void processReadRequest(final BookieProtocol.ReadRequest r, final Bookie } } - @Override - public void flushPendingResponses() { - for (Channel c : allChannels) { - c.pipeline().fireUserEventTriggered(BookieRequestHandler.EVENT_FLUSH_ALL_PENDING_RESPONSES); - } - } - public long getWaitTimeoutOnBackpressureMillis() { return waitTimeoutOnBackpressureMillis; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java index caff467db36..e50a09dba8c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java @@ -106,7 +106,6 @@ public BookieServer(ServerConfiguration conf, this.requestProcessor = new BookieRequestProcessor(conf, bookie, statsLogger.scope(SERVER_SCOPE), shFactory, allocator, nettyServer.allChannels); this.nettyServer.setRequestProcessor(this.requestProcessor); - this.bookie.setRequestProcessor(this.requestProcessor); } /** 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 47d5303bb37..41add30ca56 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 @@ -62,6 +62,7 @@ import io.netty.util.Recycler; import io.netty.util.Recycler.Handle; import io.netty.util.ReferenceCountUtil; +import io.netty.util.ReferenceCounted; import io.netty.util.concurrent.Future; import io.netty.util.concurrent.GenericFutureListener; import java.io.IOException; @@ -771,7 +772,7 @@ void forceLedger(final long ledgerId, ForceLedgerCallback cb, Object ctx) { * @param writeFlags * WriteFlags */ - void addEntry(final long ledgerId, byte[] masterKey, final long entryId, ByteBufList toSend, WriteCallback cb, + void addEntry(final long ledgerId, byte[] masterKey, final long entryId, ReferenceCounted toSend, WriteCallback cb, Object ctx, final int options, boolean allowFastFail, final EnumSet writeFlags) { Object request = null; CompletionKey completionKey = null; @@ -782,9 +783,12 @@ void addEntry(final long ledgerId, byte[] masterKey, final long entryId, ByteBuf return; } completionKey = acquireV2Key(ledgerId, entryId, OperationType.ADD_ENTRY); - request = BookieProtocol.AddRequest.create( - BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, entryId, - (short) options, masterKey, toSend); + + if (toSend instanceof ByteBuf) { + request = ((ByteBuf) toSend).retainedDuplicate(); + } else { + request = ByteBufList.clone((ByteBufList) toSend); + } } else { final long txnId = getTxnId(); completionKey = new V3CompletionKey(txnId, OperationType.ADD_ENTRY); @@ -799,11 +803,14 @@ void addEntry(final long ledgerId, byte[] masterKey, final long entryId, ByteBuf } ByteString body = null; - if (toSend.hasArray()) { - body = UnsafeByteOperations.unsafeWrap(toSend.array(), toSend.arrayOffset(), toSend.readableBytes()); + ByteBufList bufToSend = (ByteBufList) toSend; + + if (bufToSend.hasArray()) { + body = UnsafeByteOperations.unsafeWrap(bufToSend.array(), bufToSend.arrayOffset(), + bufToSend.readableBytes()); } else { - for (int i = 0; i < toSend.size(); i++) { - ByteString piece = UnsafeByteOperations.unsafeWrap(toSend.getBuffer(i).nioBuffer()); + for (int i = 0; i < bufToSend.size(); i++) { + ByteString piece = UnsafeByteOperations.unsafeWrap(bufToSend.getBuffer(i).nioBuffer()); // use ByteString.concat to avoid byte[] allocation when toSend has multiple ByteBufs body = (body == null) ? piece : body.concat(piece); } @@ -849,7 +856,7 @@ public void readLac(final long ledgerId, ReadLacCallback cb, Object ctx) { Object request = null; CompletionKey completionKey = null; if (useV2WireProtocol) { - request = new BookieProtocol.ReadRequest(BookieProtocol.CURRENT_PROTOCOL_VERSION, + request = BookieProtocol.ReadRequest.create(BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, 0, (short) 0, null); completionKey = acquireV2Key(ledgerId, 0, OperationType.READ_LAC); } else { @@ -933,7 +940,7 @@ private void readEntryInternal(final long ledgerId, Object request = null; CompletionKey completionKey = null; if (useV2WireProtocol) { - request = new BookieProtocol.ReadRequest(BookieProtocol.CURRENT_PROTOCOL_VERSION, + request = BookieProtocol.ReadRequest.create(BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, entryId, (short) flags, masterKey); completionKey = acquireV2Key(ledgerId, entryId, OperationType.READ_ENTRY); } else { @@ -1143,14 +1150,6 @@ private void writeAndFlush(final Channel channel, StringUtils.requestToString(request)); errorOut(key, BKException.Code.TooManyRequestsException); - - // If the request is a V2 add request, we retained the data's reference when creating the AddRequest - // object. To avoid the object leak, we need to release the reference if we met any errors - // before sending it. - if (request instanceof BookieProtocol.AddRequest) { - BookieProtocol.AddRequest ar = (BookieProtocol.AddRequest) request; - ar.recycle(); - } return; } @@ -1168,7 +1167,6 @@ private void writeAndFlush(final Channel channel, nettyOpLogger.registerFailedEvent(MathUtils.elapsedNanos(startTime), TimeUnit.NANOSECONDS); } }); - channel.writeAndFlush(request, promise); } catch (Throwable e) { LOG.warn("Operation {} failed", StringUtils.requestToString(request), e); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessor.java index 6935ca8be60..c44216a0283 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessor.java @@ -189,6 +189,7 @@ public String toString() { } private void recycle() { + request.recycle(); super.reset(); this.recyclerHandle.recycle(this); } 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 a97c301311f..9f931f731a2 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 @@ -24,11 +24,14 @@ import io.netty.buffer.PooledByteBufAllocator; import io.netty.buffer.Unpooled; import io.netty.util.ReferenceCountUtil; +import io.netty.util.ReferenceCounted; import io.netty.util.concurrent.FastThreadLocal; import java.security.GeneralSecurityException; import java.security.NoSuchAlgorithmException; import org.apache.bookkeeper.client.BKException.BKDigestMatchException; 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.util.ByteBufList; import org.slf4j.Logger; @@ -97,14 +100,76 @@ public static byte[] generateMasterKey(byte[] password) throws NoSuchAlgorithmEx * @param data * @return */ - public ByteBufList computeDigestAndPackageForSending(long entryId, long lastAddConfirmed, long length, - ByteBuf data) { - ByteBuf headersBuffer; + public ReferenceCounted computeDigestAndPackageForSending(long entryId, long lastAddConfirmed, long length, + ByteBuf data, byte[] masterKey, int flags) { if (this.useV2Protocol) { - headersBuffer = allocator.buffer(METADATA_LENGTH + macCodeLength); + return computeDigestAndPackageForSendingV2(entryId, lastAddConfirmed, length, data, masterKey, flags); + } else { + return computeDigestAndPackageForSendingV3(entryId, lastAddConfirmed, length, data); + } + } + + private ReferenceCounted computeDigestAndPackageForSendingV2(long entryId, long lastAddConfirmed, long length, + ByteBuf data, byte[] masterKey, int flags) { + boolean isSmallEntry = data.readableBytes() < BookieProtoEncoding.SMALL_ENTRY_SIZE_THRESHOLD; + + int headersSize = 4 // Request header + + BookieProtocol.MASTER_KEY_LENGTH // for the master key + + METADATA_LENGTH // + + macCodeLength; + int payloadSize = data.readableBytes(); + int bufferSize = 4 + headersSize + (isSmallEntry ? payloadSize : 0); + + ByteBuf buf = allocator.buffer(bufferSize, bufferSize); + buf.writeInt(headersSize + payloadSize); + buf.writeInt( + BookieProtocol.PacketHeader.toInt( + BookieProtocol.CURRENT_PROTOCOL_VERSION, BookieProtocol.ADDENTRY, (short) flags)); + buf.writeBytes(masterKey, 0, BookieProtocol.MASTER_KEY_LENGTH); + + // The checksum is computed on the next part of the buffer only + buf.readerIndex(buf.writerIndex()); + buf.writeLong(ledgerId); + buf.writeLong(entryId); + buf.writeLong(lastAddConfirmed); + buf.writeLong(length); + + // Compute checksum over the headers + int digest = update(0, buf, buf.readerIndex(), buf.readableBytes()); + + // don't unwrap slices + final ByteBuf unwrapped = data.unwrap() != null && data.unwrap() instanceof CompositeByteBuf + ? data.unwrap() : data; + ReferenceCountUtil.retain(unwrapped); + ReferenceCountUtil.safeRelease(data); + + if (unwrapped instanceof CompositeByteBuf) { + CompositeByteBuf cbb = (CompositeByteBuf) unwrapped; + for (int i = 0; i < cbb.numComponents(); i++) { + ByteBuf b = cbb.component(i); + digest = update(digest, b, b.readerIndex(), b.readableBytes()); + } + } else { + digest = update(digest, unwrapped, unwrapped.readerIndex(), unwrapped.readableBytes()); + } + + populateValueAndReset(digest, buf); + + // Reset the reader index to the beginning + buf.readerIndex(0); + + if (isSmallEntry) { + buf.writeBytes(unwrapped); + unwrapped.release(); + return buf; } else { - headersBuffer = Unpooled.buffer(METADATA_LENGTH + macCodeLength); + return ByteBufList.get(buf, unwrapped); } + } + + private ByteBufList computeDigestAndPackageForSendingV3(long entryId, long lastAddConfirmed, long length, + ByteBuf data) { + ByteBuf headersBuffer = Unpooled.buffer(METADATA_LENGTH + macCodeLength); headersBuffer.writeLong(ledgerId); headersBuffer.writeLong(entryId); headersBuffer.writeLong(lastAddConfirmed); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/Auditor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/Auditor.java index 4d7be7ace2b..739a5ff7d75 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/Auditor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/Auditor.java @@ -172,13 +172,16 @@ public Auditor(final String bookieIdentifier, bookieLedgerIndexer, hasAuditCheckTask, submitBookieCheckTask); allAuditorTasks.add(auditorBookieCheckTask); this.auditorCheckAllLedgersTask = new AuditorCheckAllLedgersTask( - conf, auditorStats, admin, ledgerManager, ledgerUnderreplicationManager, shutdownTaskHandler); + conf, auditorStats, admin, ledgerManager, + ledgerUnderreplicationManager, shutdownTaskHandler, hasAuditCheckTask); allAuditorTasks.add(auditorCheckAllLedgersTask); this.auditorPlacementPolicyCheckTask = new AuditorPlacementPolicyCheckTask( - conf, auditorStats, admin, ledgerManager, ledgerUnderreplicationManager, shutdownTaskHandler); + conf, auditorStats, admin, ledgerManager, + ledgerUnderreplicationManager, shutdownTaskHandler, hasAuditCheckTask); allAuditorTasks.add(auditorPlacementPolicyCheckTask); this.auditorReplicasCheckTask = new AuditorReplicasCheckTask( - conf, auditorStats, admin, ledgerManager, ledgerUnderreplicationManager, shutdownTaskHandler); + conf, auditorStats, admin, ledgerManager, + ledgerUnderreplicationManager, shutdownTaskHandler, hasAuditCheckTask); allAuditorTasks.add(auditorReplicasCheckTask); executor = Executors.newSingleThreadScheduledExecutor(new ThreadFactory() { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorBookieCheckTask.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorBookieCheckTask.java index 5a446e9187b..35f479638b6 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorBookieCheckTask.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorBookieCheckTask.java @@ -41,8 +41,6 @@ public class AuditorBookieCheckTask extends AuditorTask { private static final Logger LOG = LoggerFactory.getLogger(AuditorBookieCheckTask.class); private final BookieLedgerIndexer bookieLedgerIndexer; - private final BiConsumer hasAuditCheckTask; - private final AtomicBoolean hasTask = new AtomicBoolean(false); private final BiConsumer submitCheckTask; public AuditorBookieCheckTask(ServerConfiguration conf, @@ -55,17 +53,14 @@ public AuditorBookieCheckTask(ServerConfiguration conf, BiConsumer hasAuditCheckTask, BiConsumer submitCheckTask) { super(conf, auditorStats, admin, ledgerManager, - ledgerUnderreplicationManager, shutdownTaskHandler); + ledgerUnderreplicationManager, shutdownTaskHandler, hasAuditCheckTask); this.bookieLedgerIndexer = bookieLedgerIndexer; - this.hasAuditCheckTask = hasAuditCheckTask; this.submitCheckTask = submitCheckTask; } @Override protected void runTask() { - hasTask.set(false); - hasAuditCheckTask.accept(hasTask, null); - if (!hasTask.get()) { + if (!hasBookieCheckTask()) { startAudit(true); } else { // if due to a lost bookie an audit task was scheduled, diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorCheckAllLedgersTask.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorCheckAllLedgersTask.java index bddef5a88f2..73ca36cd752 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorCheckAllLedgersTask.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorCheckAllLedgersTask.java @@ -27,6 +27,8 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiConsumer; import java.util.stream.Collectors; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; @@ -57,10 +59,11 @@ public class AuditorCheckAllLedgersTask extends AuditorTask { BookKeeperAdmin admin, LedgerManager ledgerManager, LedgerUnderreplicationManager ledgerUnderreplicationManager, - ShutdownTaskHandler shutdownTaskHandler) + ShutdownTaskHandler shutdownTaskHandler, + BiConsumer hasAuditCheckTask) throws UnavailableException { super(conf, auditorStats, admin, ledgerManager, - ledgerUnderreplicationManager, shutdownTaskHandler); + ledgerUnderreplicationManager, shutdownTaskHandler, hasAuditCheckTask); if (conf.getAuditorMaxNumberOfConcurrentOpenLedgerOperations() <= 0) { LOG.error("auditorMaxNumberOfConcurrentOpenLedgerOperations should be greater than 0"); @@ -89,6 +92,11 @@ public Thread newThread(Runnable r) { @Override protected void runTask() { + if (hasBookieCheckTask()) { + LOG.info("Audit bookie task already scheduled; skipping periodic all ledgers check task"); + return; + } + Stopwatch stopwatch = Stopwatch.createStarted(); boolean checkSuccess = false; try { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTask.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTask.java index 74442b655a4..04fbc84f3e1 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTask.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTask.java @@ -25,7 +25,9 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; import lombok.Getter; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeperAdmin; @@ -58,9 +60,10 @@ public class AuditorPlacementPolicyCheckTask extends AuditorTask { BookKeeperAdmin admin, LedgerManager ledgerManager, LedgerUnderreplicationManager ledgerUnderreplicationManager, - ShutdownTaskHandler shutdownTaskHandler) { + ShutdownTaskHandler shutdownTaskHandler, + BiConsumer hasAuditCheckTask) { super(conf, auditorStats, admin, ledgerManager, - ledgerUnderreplicationManager, shutdownTaskHandler); + ledgerUnderreplicationManager, shutdownTaskHandler, hasAuditCheckTask); this.underreplicatedLedgerRecoveryGracePeriod = conf.getUnderreplicatedLedgerRecoveryGracePeriod(); this.numOfLedgersFoundNotAdheringInPlacementPolicyCheck = new AtomicInteger(0); this.numOfLedgersFoundSoftlyAdheringInPlacementPolicyCheck = new AtomicInteger(0); @@ -70,6 +73,11 @@ public class AuditorPlacementPolicyCheckTask extends AuditorTask { @Override protected void runTask() { + if (hasBookieCheckTask()) { + LOG.info("Audit bookie task already scheduled; skipping periodic placement policy check task"); + return; + } + try { if (!isLedgerReplicationEnabled()) { LOG.info("Ledger replication disabled, skipping placementPolicyCheck"); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTask.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTask.java index 7ce66883b6e..b9e47c838dd 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTask.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTask.java @@ -34,6 +34,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; import org.apache.bookkeeper.client.BKException; @@ -69,9 +70,10 @@ public class AuditorReplicasCheckTask extends AuditorTask { AuditorStats auditorStats, BookKeeperAdmin admin, LedgerManager ledgerManager, LedgerUnderreplicationManager ledgerUnderreplicationManager, - ShutdownTaskHandler shutdownTaskHandler) { + ShutdownTaskHandler shutdownTaskHandler, + BiConsumer hasAuditCheckTask) { super(conf, auditorStats, admin, ledgerManager, - ledgerUnderreplicationManager, shutdownTaskHandler); + ledgerUnderreplicationManager, shutdownTaskHandler, hasAuditCheckTask); this.zkOpTimeoutMs = conf.getZkTimeout() * 2; this.numLedgersFoundHavingNoReplicaOfAnEntry = new AtomicInteger(0); this.numLedgersFoundHavingLessThanAQReplicasOfAnEntry = new AtomicInteger(0); @@ -80,6 +82,11 @@ public class AuditorReplicasCheckTask extends AuditorTask { @Override protected void runTask() { + if (hasBookieCheckTask()) { + LOG.info("Audit bookie task already scheduled; skipping periodic replicas check task"); + return; + } + try { if (!ledgerUnderreplicationManager.isLedgerReplicationEnabled()) { LOG.info("Ledger replication disabled, skipping replicasCheck task."); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorTask.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorTask.java index 3306f4d0cb9..895cc70dea2 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorTask.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorTask.java @@ -24,7 +24,9 @@ import java.util.List; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.LongAdder; +import java.util.function.BiConsumer; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeperAdmin; @@ -46,19 +48,23 @@ abstract class AuditorTask implements Runnable { protected LedgerManager ledgerManager; protected LedgerUnderreplicationManager ledgerUnderreplicationManager; private final ShutdownTaskHandler shutdownTaskHandler; + private final BiConsumer hasAuditCheckTask; + private final AtomicBoolean hasTask = new AtomicBoolean(false); AuditorTask(ServerConfiguration conf, AuditorStats auditorStats, BookKeeperAdmin admin, LedgerManager ledgerManager, LedgerUnderreplicationManager ledgerUnderreplicationManager, - ShutdownTaskHandler shutdownTaskHandler) { + ShutdownTaskHandler shutdownTaskHandler, + BiConsumer hasAuditCheckTask) { this.conf = conf; this.auditorStats = auditorStats; this.admin = admin; this.ledgerManager = ledgerManager; this.ledgerUnderreplicationManager = ledgerUnderreplicationManager; this.shutdownTaskHandler = shutdownTaskHandler; + this.hasAuditCheckTask = hasAuditCheckTask; } @Override @@ -141,6 +147,12 @@ protected void submitShutdownTask() { public abstract void shutdown(); + protected boolean hasBookieCheckTask() { + hasTask.set(false); + hasAuditCheckTask.accept(hasTask, null); + return hasTask.get(); + } + /** * ShutdownTaskHandler used to shutdown auditor executor. */ diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/EmbeddedServer.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/EmbeddedServer.java index b8d4f537890..bfa80246f56 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/EmbeddedServer.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/EmbeddedServer.java @@ -302,7 +302,7 @@ public EmbeddedServer build() throws Exception { // 2. Build metadata driver if (metadataDriver == null) { - if (ledgerManagerFactory == null && registrationManager == null) { + if (ledgerManagerFactory == null || registrationManager == null) { metadataDriver = BookieResources.createMetadataDriver(conf.getServerConf(), rootStatsLogger); serverBuilder.addComponent(new AutoCloseableLifecycleComponent("metadataDriver", metadataDriver)); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java index 4c7b787405e..823cf486524 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/BKHttpServiceProvider.java @@ -64,6 +64,7 @@ import org.apache.bookkeeper.server.http.service.SuspendCompactionService; import org.apache.bookkeeper.server.http.service.TriggerAuditService; import org.apache.bookkeeper.server.http.service.TriggerGCService; +import org.apache.bookkeeper.server.http.service.TriggerLocationCompactService; import org.apache.bookkeeper.server.http.service.WhoIsAuditorService; import org.apache.bookkeeper.stats.StatsProvider; import org.apache.zookeeper.KeeperException; @@ -235,6 +236,8 @@ public HttpEndpointService provideHttpEndpointService(ApiType type) { return new SuspendCompactionService(bookieServer); case RESUME_GC_COMPACTION: return new ResumeCompactionService(bookieServer); + case TRIGGER_ENTRY_LOCATION_COMPACT: + return new TriggerLocationCompactService(bookieServer); // autorecovery case AUTORECOVERY_STATUS: diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/TriggerLocationCompactService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/TriggerLocationCompactService.java new file mode 100644 index 00000000000..95b7f277506 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/TriggerLocationCompactService.java @@ -0,0 +1,143 @@ +/* + * 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.server.http.service; + +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.bookkeeper.bookie.LedgerStorage; +import org.apache.bookkeeper.common.util.JsonUtil; +import org.apache.bookkeeper.http.HttpServer; +import org.apache.bookkeeper.http.service.HttpEndpointService; +import org.apache.bookkeeper.http.service.HttpServiceRequest; +import org.apache.bookkeeper.http.service.HttpServiceResponse; +import org.apache.bookkeeper.proto.BookieServer; +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * HttpEndpointService that handle force trigger entry location compact requests. + * + *

The PUT method will trigger entry location compact on current bookie. + * + *

The GET method will get the entry location compact running or not. + * Output would be like: + * { + * "/data1/bookkeeper/ledgers/current/locations" : "false", + * "/data2/bookkeeper/ledgers/current/locations" : "true", + * } + */ + +public class TriggerLocationCompactService implements HttpEndpointService { + + static final Logger LOG = LoggerFactory.getLogger(TriggerLocationCompactService.class); + + private final BookieServer bookieServer; + private final List entryLocationDBPath; + + public TriggerLocationCompactService(BookieServer bookieServer) { + this.bookieServer = checkNotNull(bookieServer); + this.entryLocationDBPath = bookieServer.getBookie().getLedgerStorage().getEntryLocationDBPath(); + } + + @Override + public HttpServiceResponse handle(HttpServiceRequest request) throws Exception { + HttpServiceResponse response = new HttpServiceResponse(); + LedgerStorage ledgerStorage = bookieServer.getBookie().getLedgerStorage(); + + if (HttpServer.Method.PUT.equals(request.getMethod())) { + String requestBody = request.getBody(); + String output = "Not trigger Entry Location RocksDB compact."; + + if (StringUtils.isBlank(requestBody)) { + output = "Empty request body"; + response.setBody(output); + response.setCode(HttpServer.StatusCode.BAD_REQUEST); + return response; + } + + try { + @SuppressWarnings("unchecked") + Map configMap = JsonUtil.fromJson(requestBody, HashMap.class); + Boolean isEntryLocationCompact = (Boolean) configMap + .getOrDefault("entryLocationRocksDBCompact", false); + String entryLocations = (String) configMap.getOrDefault("entryLocations", ""); + + if (!isEntryLocationCompact) { + // If entryLocationRocksDBCompact is false, doing nothing. + response.setBody(output); + response.setCode(HttpServer.StatusCode.OK); + return response; + } + if (StringUtils.isNotBlank(entryLocations)) { + // Specified trigger RocksDB compact entryLocations. + Set locations = Sets.newHashSet(entryLocations.trim().split(",")); + if (CollectionUtils.isSubCollection(locations, entryLocationDBPath)) { + ledgerStorage.entryLocationCompact(Lists.newArrayList(locations)); + output = String.format("Triggered entry Location RocksDB: %s compact on bookie:%s.", + entryLocations, bookieServer.getBookieId()); + response.setCode(HttpServer.StatusCode.OK); + } else { + output = String.format("Specified trigger compact entryLocations: %s is invalid. " + + "Bookie entry location RocksDB path: %s.", entryLocations, entryLocationDBPath); + response.setCode(HttpServer.StatusCode.BAD_REQUEST); + } + } else { + // Not specified trigger compact entryLocations, trigger compact for all entry location. + ledgerStorage.entryLocationCompact(); + output = "Triggered entry Location RocksDB compact on bookie:" + bookieServer.getBookieId(); + response.setCode(HttpServer.StatusCode.OK); + } + } catch (JsonUtil.ParseJsonException ex) { + output = ex.getMessage(); + response.setCode(HttpServer.StatusCode.BAD_REQUEST); + LOG.warn("Trigger entry location index RocksDB compact failed, caused by: " + ex.getMessage()); + } + + String jsonResponse = JsonUtil.toJson(output); + if (LOG.isDebugEnabled()) { + LOG.debug("output body:" + jsonResponse); + } + response.setBody(jsonResponse); + return response; + } else if (HttpServer.Method.GET == request.getMethod()) { + Map compactStatus = ledgerStorage.isEntryLocationCompacting(entryLocationDBPath); + String jsonResponse = JsonUtil.toJson(compactStatus); + if (LOG.isDebugEnabled()) { + LOG.debug("output body:" + jsonResponse); + } + response.setBody(jsonResponse); + response.setCode(HttpServer.StatusCode.OK); + return response; + } else { + response.setCode(HttpServer.StatusCode.METHOD_NOT_ALLOWED); + response.setBody("Not found method. Should be PUT to trigger entry location compact," + + " Or GET to get entry location compact state."); + return response; + } + } +} \ No newline at end of file diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/FileInfoBackingCacheTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/FileInfoBackingCacheTest.java index e67d877e738..e7d27ea8a80 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/FileInfoBackingCacheTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/FileInfoBackingCacheTest.java @@ -26,6 +26,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.io.File; import java.io.IOException; +import java.security.SecureRandom; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; @@ -80,7 +81,7 @@ public void tearDown() throws Exception { } } - @Test + @Test(timeout = 30000) public void basicTest() throws Exception { FileInfoBackingCache cache = new FileInfoBackingCache( (ledgerId, createIfNotFound) -> { @@ -108,7 +109,7 @@ public void basicTest() throws Exception { Assert.assertEquals(fi.getLf(), fi4.getLf()); } - @Test(expected = IOException.class) + @Test(expected = IOException.class, timeout = 30000) public void testNoKey() throws Exception { FileInfoBackingCache cache = new FileInfoBackingCache( (ledgerId, createIfNotFound) -> { @@ -122,7 +123,7 @@ public void testNoKey() throws Exception { * Of course this can't prove they don't exist, but * try to shake them out none the less. */ - @Test + @Test(timeout = 30000) public void testForDeadlocks() throws Exception { int numRunners = 20; int maxLedgerId = 10; @@ -184,7 +185,7 @@ public void testForDeadlocks() throws Exception { } } - @Test + @Test(timeout = 30000) public void testRefCountRace() throws Exception { AtomicBoolean done = new AtomicBoolean(false); FileInfoBackingCache cache = new FileInfoBackingCache( @@ -229,9 +230,10 @@ private void guavaEvictionListener(RemovalNotification not notification.getValue().release(); } - @Test + @Test(timeout = 30000) public void testRaceGuavaEvictAndReleaseBeforeRetain() throws Exception { AtomicBoolean done = new AtomicBoolean(false); + Random random = new SecureRandom(); FileInfoBackingCache cache = new FileInfoBackingCache( (ledgerId, createIfNotFound) -> { File f = new File(baseDir, String.valueOf(ledgerId)); @@ -254,9 +256,9 @@ public void testRaceGuavaEvictAndReleaseBeforeRetain() throws Exception { do { fi = guavaCache.get( - i, () -> cache.loadFileInfo(i, masterKey)); + i, () -> cache.loadFileInfo(i, masterKey)); allFileInfos.add(fi); - Thread.sleep(100); + Thread.sleep(random.nextInt(100)); } while (!fi.tryRetain()); Assert.assertFalse(fi.isClosed()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GarbageCollectorThreadTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GarbageCollectorThreadTest.java index 31828423a43..ebe07ce2303 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GarbageCollectorThreadTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GarbageCollectorThreadTest.java @@ -30,6 +30,7 @@ import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.any; @@ -238,4 +239,138 @@ private void testExtractMetaFromEntryLogs(EntryLogger entryLogger, File ledgerDi assertTrue(entryLogMetaMap.isEmpty()); assertFalse(entryLogger.logExists(logId3)); } + + @Test + public void testCompactionWithFileSizeCheck() throws Exception { + File ledgerDir = tmpDirs.createNew("testFileSize", "ledgers"); + EntryLogger entryLogger = newLegacyEntryLogger(20000, ledgerDir); + + MockLedgerStorage storage = new MockLedgerStorage(); + MockLedgerManager lm = new MockLedgerManager(); + + GarbageCollectorThread gcThread = new GarbageCollectorThread( + TestBKConfiguration.newServerConfiguration().setUseTargetEntryLogSizeForGc(true), lm, + newDirsManager(ledgerDir), + storage, entryLogger, NullStatsLogger.INSTANCE); + + // Add entries. + // Ledger 1 is on first entry log + // Ledger 2 spans first, second and third entry log + // Ledger 3 is on the third entry log (which is still active when extract meta) + long loc1 = entryLogger.addEntry(1L, makeEntry(1L, 1L, 5000)); + long loc2 = entryLogger.addEntry(2L, makeEntry(2L, 1L, 5000)); + assertThat(logIdFromLocation(loc2), equalTo(logIdFromLocation(loc1))); + long loc3 = entryLogger.addEntry(2L, makeEntry(2L, 2L, 15000)); + assertThat(logIdFromLocation(loc3), greaterThan(logIdFromLocation(loc2))); + long loc4 = entryLogger.addEntry(2L, makeEntry(2L, 3L, 15000)); + assertThat(logIdFromLocation(loc4), greaterThan(logIdFromLocation(loc3))); + long loc5 = entryLogger.addEntry(3L, makeEntry(3L, 1L, 1000)); + assertThat(logIdFromLocation(loc5), equalTo(logIdFromLocation(loc4))); + long loc6 = entryLogger.addEntry(3L, makeEntry(3L, 2L, 5000)); + + long logId1 = logIdFromLocation(loc2); + long logId2 = logIdFromLocation(loc3); + long logId3 = logIdFromLocation(loc5); + long logId4 = logIdFromLocation(loc6); + entryLogger.flush(); + + storage.setMasterKey(1L, new byte[0]); + storage.setMasterKey(2L, new byte[0]); + storage.setMasterKey(3L, new byte[0]); + + assertThat(entryLogger.getFlushedLogIds(), containsInAnyOrder(logId1, logId2, logId3)); + assertTrue(entryLogger.logExists(logId1)); + assertTrue(entryLogger.logExists(logId2)); + assertTrue(entryLogger.logExists(logId3)); + assertTrue(entryLogger.logExists(logId4)); + + // all ledgers exist, nothing should disappear + final EntryLogMetadataMap entryLogMetaMap = gcThread.getEntryLogMetaMap(); + gcThread.extractMetaFromEntryLogs(); + + assertThat(entryLogger.getFlushedLogIds(), containsInAnyOrder(logId1, logId2, logId3)); + assertTrue(entryLogMetaMap.containsKey(logId1)); + assertTrue(entryLogMetaMap.containsKey(logId2)); + assertTrue(entryLogger.logExists(logId3)); + + storage.deleteLedger(1); + // only logId 1 will be compacted. + gcThread.runWithFlags(true, true, false); + + // logId1 and logId2 should be compacted + assertFalse(entryLogger.logExists(logId1)); + assertTrue(entryLogger.logExists(logId2)); + assertTrue(entryLogger.logExists(logId3)); + assertFalse(entryLogMetaMap.containsKey(logId1)); + assertTrue(entryLogMetaMap.containsKey(logId2)); + + assertEquals(1, storage.getUpdatedLocations().size()); + + EntryLocation location2 = storage.getUpdatedLocations().get(0); + assertEquals(2, location2.getLedger()); + assertEquals(1, location2.getEntry()); + assertEquals(logIdFromLocation(location2.getLocation()), logId4); + } + + @Test + public void testCompactionWithoutFileSizeCheck() throws Exception { + File ledgerDir = tmpDirs.createNew("testFileSize", "ledgers"); + EntryLogger entryLogger = newLegacyEntryLogger(20000, ledgerDir); + + MockLedgerStorage storage = new MockLedgerStorage(); + MockLedgerManager lm = new MockLedgerManager(); + + GarbageCollectorThread gcThread = new GarbageCollectorThread( + TestBKConfiguration.newServerConfiguration(), lm, + newDirsManager(ledgerDir), + storage, entryLogger, NullStatsLogger.INSTANCE); + + // Add entries. + // Ledger 1 is on first entry log + // Ledger 2 spans first, second and third entry log + // Ledger 3 is on the third entry log (which is still active when extract meta) + long loc1 = entryLogger.addEntry(1L, makeEntry(1L, 1L, 5000)); + long loc2 = entryLogger.addEntry(2L, makeEntry(2L, 1L, 5000)); + assertThat(logIdFromLocation(loc2), equalTo(logIdFromLocation(loc1))); + long loc3 = entryLogger.addEntry(2L, makeEntry(2L, 2L, 15000)); + assertThat(logIdFromLocation(loc3), greaterThan(logIdFromLocation(loc2))); + long loc4 = entryLogger.addEntry(2L, makeEntry(2L, 3L, 15000)); + assertThat(logIdFromLocation(loc4), greaterThan(logIdFromLocation(loc3))); + long loc5 = entryLogger.addEntry(3L, makeEntry(3L, 1L, 1000)); + assertThat(logIdFromLocation(loc5), equalTo(logIdFromLocation(loc4))); + + long logId1 = logIdFromLocation(loc2); + long logId2 = logIdFromLocation(loc3); + long logId3 = logIdFromLocation(loc5); + entryLogger.flush(); + + storage.setMasterKey(1L, new byte[0]); + storage.setMasterKey(2L, new byte[0]); + storage.setMasterKey(3L, new byte[0]); + + assertThat(entryLogger.getFlushedLogIds(), containsInAnyOrder(logId1, logId2)); + assertTrue(entryLogger.logExists(logId1)); + assertTrue(entryLogger.logExists(logId2)); + assertTrue(entryLogger.logExists(logId3)); + + // all ledgers exist, nothing should disappear + final EntryLogMetadataMap entryLogMetaMap = gcThread.getEntryLogMetaMap(); + gcThread.extractMetaFromEntryLogs(); + + assertThat(entryLogger.getFlushedLogIds(), containsInAnyOrder(logId1, logId2)); + assertTrue(entryLogMetaMap.containsKey(logId1)); + assertTrue(entryLogMetaMap.containsKey(logId2)); + assertTrue(entryLogger.logExists(logId3)); + + gcThread.runWithFlags(true, true, false); + + assertTrue(entryLogger.logExists(logId1)); + assertTrue(entryLogger.logExists(logId2)); + assertTrue(entryLogger.logExists(logId3)); + assertTrue(entryLogMetaMap.containsKey(logId1)); + assertTrue(entryLogMetaMap.containsKey(logId2)); + + assertEquals(0, storage.getUpdatedLocations().size()); + } + } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpgradeTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpgradeTest.java index b259dfd9016..e484f91133b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpgradeTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UpgradeTest.java @@ -21,6 +21,7 @@ package org.apache.bookkeeper.bookie; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -36,6 +37,7 @@ import java.nio.ByteBuffer; import java.nio.channels.FileChannel; import java.util.Arrays; +import java.util.List; import org.apache.bookkeeper.client.ClientUtil; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.conf.ServerConfiguration; @@ -62,6 +64,28 @@ public UpgradeTest() { super(0); } + static void writeLedgerDirWithIndexDir(File ledgerDir, + File indexDir, + byte[] masterKey) + throws Exception { + long ledgerId = 1; + + File fn = new File(indexDir, IndexPersistenceMgr.getLedgerName(ledgerId)); + fn.getParentFile().mkdirs(); + FileInfo fi = new FileInfo(fn, masterKey, FileInfo.CURRENT_HEADER_VERSION); + // force creation of index file + fi.write(new ByteBuffer[]{ ByteBuffer.allocate(0) }, 0); + fi.close(true); + + long logId = 0; + ByteBuffer logfileHeader = ByteBuffer.allocate(1024); + logfileHeader.put("BKLO".getBytes()); + FileChannel logfile = new RandomAccessFile( + new File(ledgerDir, Long.toHexString(logId) + ".log"), "rw").getChannel(); + logfile.write((ByteBuffer) logfileHeader.clear()); + logfile.close(); + } + static void writeLedgerDir(File dir, byte[] masterKey) throws Exception { @@ -122,6 +146,12 @@ static File initV1LedgerDirectory(File d) throws Exception { return d; } + static File initV1LedgerDirectoryWithIndexDir(File ledgerDir, + File indexDir) throws Exception { + writeLedgerDirWithIndexDir(ledgerDir, indexDir, "foobar".getBytes()); + return ledgerDir; + } + static void createVersion2File(File dir) throws Exception { File versionFile = new File(dir, "VERSION"); @@ -148,12 +178,21 @@ static File initV2LedgerDirectory(File d) throws Exception { return d; } - private static void testUpgradeProceedure(String zkServers, String journalDir, String ledgerDir) throws Exception { + static File initV2LedgerDirectoryWithIndexDir(File ledgerDir, File indexDir) throws Exception { + initV1LedgerDirectoryWithIndexDir(ledgerDir, indexDir); + createVersion2File(ledgerDir); + createVersion2File(indexDir); + return ledgerDir; + } + + private static void testUpgradeProceedure(String zkServers, String journalDir, String ledgerDir, String indexDir) + throws Exception { ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); conf.setMetadataServiceUri("zk://" + zkServers + "/ledgers"); conf.setJournalDirName(journalDir) - .setLedgerDirNames(new String[] { ledgerDir }) - .setBookiePort(bookiePort); + .setLedgerDirNames(new String[]{ledgerDir}) + .setIndexDirName(new String[]{indexDir}) + .setBookiePort(bookiePort); Bookie b = null; try (MetadataBookieDriver metadataDriver = BookieResources.createMetadataDriver( @@ -211,21 +250,62 @@ private static void testUpgradeProceedure(String zkServers, String journalDir, S public void testUpgradeV1toCurrent() throws Exception { File journalDir = initV1JournalDirectory(tmpDirs.createNew("bookie", "journal")); File ledgerDir = initV1LedgerDirectory(tmpDirs.createNew("bookie", "ledger")); - testUpgradeProceedure(zkUtil.getZooKeeperConnectString(), journalDir.getPath(), ledgerDir.getPath()); + testUpgradeProceedure(zkUtil.getZooKeeperConnectString(), journalDir.getPath(), + ledgerDir.getPath(), ledgerDir.getPath()); + } + + @Test + public void testUpgradeV1toCurrentWithIndexDir() throws Exception { + File journalDir = initV1JournalDirectory(tmpDirs.createNew("bookie", "journal")); + File indexDir = tmpDirs.createNew("bookie", "index"); + File ledgerDir = initV1LedgerDirectoryWithIndexDir( + tmpDirs.createNew("bookie", "ledger"), indexDir); + testUpgradeProceedure(zkUtil.getZooKeeperConnectString(), journalDir.getPath(), + ledgerDir.getPath(), indexDir.getPath()); } @Test public void testUpgradeV2toCurrent() throws Exception { File journalDir = initV2JournalDirectory(tmpDirs.createNew("bookie", "journal")); File ledgerDir = initV2LedgerDirectory(tmpDirs.createNew("bookie", "ledger")); - testUpgradeProceedure(zkUtil.getZooKeeperConnectString(), journalDir.getPath(), ledgerDir.getPath()); + File indexDir = tmpDirs.createNew("bookie", "index"); + testUpgradeProceedure(zkUtil.getZooKeeperConnectString(), journalDir.getPath(), + ledgerDir.getPath(), indexDir.getPath()); + } + + @Test + public void testUpgradeV2toCurrentWithIndexDir() throws Exception { + File journalDir = initV2JournalDirectory(tmpDirs.createNew("bookie", "journal")); + File indexDir = tmpDirs.createNew("bookie", "index"); + File ledgerDir = initV2LedgerDirectoryWithIndexDir( + tmpDirs.createNew("bookie", "ledger"), indexDir); + testUpgradeProceedure(zkUtil.getZooKeeperConnectString(), journalDir.getPath(), + ledgerDir.getPath(), indexDir.getPath()); } @Test public void testUpgradeCurrent() throws Exception { + testUpgradeCurrent(false); + } + + @Test + public void testUpgradeCurrentWithIndexDir() throws Exception { + testUpgradeCurrent(true); + } + + public void testUpgradeCurrent(boolean hasIndexDir) throws Exception { File journalDir = initV2JournalDirectory(tmpDirs.createNew("bookie", "journal")); - File ledgerDir = initV2LedgerDirectory(tmpDirs.createNew("bookie", "ledger")); - testUpgradeProceedure(zkUtil.getZooKeeperConnectString(), journalDir.getPath(), ledgerDir.getPath()); + File ledgerDir = tmpDirs.createNew("bookie", "ledger"); + File indexDir = ledgerDir; + if (hasIndexDir) { + indexDir = tmpDirs.createNew("bookie", "index"); + initV2LedgerDirectoryWithIndexDir(ledgerDir, indexDir); + } else { + initV2LedgerDirectory(ledgerDir); + } + + testUpgradeProceedure(zkUtil.getZooKeeperConnectString(), journalDir.getPath(), + ledgerDir.getPath(), indexDir.getPath()); // Upgrade again ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); @@ -278,4 +358,43 @@ public void testCommandLine() throws Exception { System.setErr(origerr); } } + + @Test + public void testFSUGetAllDirectories() throws Exception { + ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); + final File journalDir = tmpDirs.createNew("bookie", "journal"); + final File ledgerDir1 = tmpDirs.createNew("bookie", "ledger"); + final File ledgerDir2 = tmpDirs.createNew("bookie", "ledger"); + + // test1 + conf.setJournalDirName(journalDir.getPath()) + .setLedgerDirNames(new String[]{ledgerDir1.getPath(), ledgerDir2.getPath()}) + .setIndexDirName(new String[]{ledgerDir1.getPath(), ledgerDir2.getPath()}); + List allDirectories = FileSystemUpgrade.getAllDirectories(conf); + assertEquals(3, allDirectories.size()); + + // test2 + conf.setJournalDirName(journalDir.getPath()) + .setLedgerDirNames(new String[]{ledgerDir1.getPath(), ledgerDir2.getPath()}) + .setIndexDirName(new String[]{ledgerDir2.getPath(), ledgerDir1.getPath()}); + allDirectories = FileSystemUpgrade.getAllDirectories(conf); + assertEquals(3, allDirectories.size()); + + final File indexDir1 = tmpDirs.createNew("bookie", "index"); + final File indexDir2 = tmpDirs.createNew("bookie", "index"); + + // test3 + conf.setJournalDirName(journalDir.getPath()) + .setLedgerDirNames(new String[]{ledgerDir1.getPath(), ledgerDir2.getPath()}) + .setIndexDirName(new String[]{indexDir1.getPath(), indexDir2.getPath()}); + allDirectories = FileSystemUpgrade.getAllDirectories(conf); + assertEquals(5, allDirectories.size()); + + // test4 + conf.setJournalDirName(journalDir.getPath()) + .setLedgerDirNames(new String[]{ledgerDir1.getPath(), ledgerDir2.getPath()}) + .setIndexDirName(new String[]{indexDir2.getPath(), indexDir1.getPath()}); + allDirectories = FileSystemUpgrade.getAllDirectories(conf); + assertEquals(5, allDirectories.size()); + } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDBTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDBTest.java index 6a0e00b0c10..2ef3e010f8b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDBTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDBTest.java @@ -54,6 +54,7 @@ public void testRocksDBInitiateWithBookieConfiguration() throws Exception { assertEquals(64 * 1024 * 1024, options.writeBufferSize()); assertEquals(4, options.maxWriteBufferNumber()); assertEquals(256 * 1024 * 1024, options.maxBytesForLevelBase()); + assertEquals(true, options.levelCompactionDynamicLevelBytes()); rocksDB.close(); } @@ -78,6 +79,7 @@ public void testRocksDBInitiateWithConfigurationFile() throws Exception { assertEquals(CompressionType.LZ4_COMPRESSION, familyOptions.compressionType()); assertEquals(1024, familyOptions.writeBufferSize()); assertEquals(1, familyOptions.maxWriteBufferNumber()); + assertEquals(true, familyOptions.levelCompactionDynamicLevelBytes()); rocksDB.close(); } @@ -113,4 +115,20 @@ public void testReadChecksumTypeFromConfigurationFile() throws Exception { // After the PR: https://github.com/facebook/rocksdb/pull/10826 merge, we can turn on this test. assertEquals(ChecksumType.kxxHash, ((BlockBasedTableConfig) familyOptions.tableFormatConfig()).checksumType()); } + + @Test + public void testLevelCompactionDynamicLevelBytesFromConfigurationFile() throws Exception { + ServerConfiguration configuration = new ServerConfiguration(); + URL url = getClass().getClassLoader().getResource("conf/entry_location_rocksdb.conf"); + configuration.setEntryLocationRocksdbConf(url.getPath()); + File tmpDir = Files.createTempDirectory("bk-kv-rocksdbtest-file").toFile(); + Files.createDirectory(Paths.get(tmpDir.toString(), "subDir")); + KeyValueStorageRocksDB rocksDB = new KeyValueStorageRocksDB(tmpDir.toString(), "subDir", + KeyValueStorageFactory.DbConfigType.EntryLocation, configuration); + assertNotNull(rocksDB.getColumnFamilyDescriptors()); + + List columnFamilyDescriptorList = rocksDB.getColumnFamilyDescriptors(); + ColumnFamilyOptions familyOptions = columnFamilyDescriptorList.get(0).getOptions(); + assertEquals(true, familyOptions.levelCompactionDynamicLevelBytes()); + } } 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 d24022e5042..3f8af53c133 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 @@ -27,8 +27,8 @@ 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.MockBookieClient; import org.apache.bookkeeper.proto.checksum.DigestManager; -import org.apache.bookkeeper.util.ByteBufList; import org.apache.bookkeeper.versioning.Versioned; /** @@ -48,8 +48,8 @@ public static ByteBuf generatePacket(long ledgerId, long entryId, long lastAddCo int offset, int len) throws GeneralSecurityException { DigestManager dm = DigestManager.instantiate(ledgerId, new byte[2], DigestType.CRC32, UnpooledByteBufAllocator.DEFAULT, true); - return ByteBufList.coalesce(dm.computeDigestAndPackageForSending(entryId, lastAddConfirmed, length, - Unpooled.wrappedBuffer(data, offset, len))); + return MockBookieClient.copyDataWithSkipHeader(dm.computeDigestAndPackageForSending(entryId, lastAddConfirmed, + length, Unpooled.wrappedBuffer(data, offset, len), new byte[20], 0)); } /** diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java index 2f5ae8f5ba0..252fb83330c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/HandleFailuresTest.java @@ -54,7 +54,7 @@ public class HandleFailuresTest { private static final BookieId b4 = new BookieSocketAddress("b4", 3181).toBookieId(); private static final BookieId b5 = new BookieSocketAddress("b5", 3181).toBookieId(); - @Test + @Test(timeout = 30000) public void testChangeTriggeredOneTimeForOneFailure() throws Exception { MockClientContext clientCtx = MockClientContext.create(); Versioned md = ClientUtil.setupLedger(clientCtx, 10L, @@ -77,7 +77,7 @@ public void testChangeTriggeredOneTimeForOneFailure() throws Exception { Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b4, b2, b3)); } - @Test + @Test(timeout = 30000) public void testSecondFailureOccursWhileFirstBeingHandled() throws Exception { MockClientContext clientCtx = MockClientContext.create(); Versioned md = ClientUtil.setupLedger(clientCtx, 10L, @@ -125,7 +125,7 @@ public void testSecondFailureOccursWhileFirstBeingHandled() throws Exception { Assert.assertTrue(lh.getLedgerMetadata().getAllEnsembles().get(0L).contains(b5)); } - @Test + @Test(timeout = 30000) public void testHandlingFailuresOneBookieFailsImmediately() throws Exception { MockClientContext clientCtx = MockClientContext.create(); Versioned md = ClientUtil.setupLedger(clientCtx, 10L, @@ -145,7 +145,7 @@ public void testHandlingFailuresOneBookieFailsImmediately() throws Exception { Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b4, b2, b3)); } - @Test + @Test(timeout = 30000) public void testHandlingFailuresOneBookieFailsAfterOneEntry() throws Exception { MockClientContext clientCtx = MockClientContext.create(); Versioned md = ClientUtil.setupLedger(clientCtx, 10L, @@ -168,7 +168,7 @@ public void testHandlingFailuresOneBookieFailsAfterOneEntry() throws Exception { Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), 1L); } - @Test + @Test(timeout = 30000) public void testHandlingFailuresMultipleBookieFailImmediatelyNotEnoughToReplace() throws Exception { MockClientContext clientCtx = MockClientContext.create(); Versioned md = ClientUtil.setupLedger(clientCtx, 10L, @@ -194,7 +194,7 @@ public void testHandlingFailuresMultipleBookieFailImmediatelyNotEnoughToReplace( } } - @Test + @Test(timeout = 30000) public void testHandlingFailuresMultipleBookieFailAfterOneEntryNotEnoughToReplace() throws Exception { MockClientContext clientCtx = MockClientContext.create(); Versioned md = ClientUtil.setupLedger(clientCtx, 10L, @@ -222,7 +222,7 @@ public void testHandlingFailuresMultipleBookieFailAfterOneEntryNotEnoughToReplac } } - @Test + @Test(timeout = 30000) public void testClientClosesWhileFailureHandlerInProgress() throws Exception { MockClientContext clientCtx = MockClientContext.create(); Versioned md = ClientUtil.setupLedger(clientCtx, 10L, @@ -264,7 +264,7 @@ public void testClientClosesWhileFailureHandlerInProgress() throws Exception { Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), LedgerHandle.INVALID_ENTRY_ID); } - @Test + @Test(timeout = 30000) public void testMetadataSetToClosedDuringFailureHandler() throws Exception { MockClientContext clientCtx = MockClientContext.create(); Versioned md = ClientUtil.setupLedger(clientCtx, 10L, @@ -308,7 +308,7 @@ public void testMetadataSetToClosedDuringFailureHandler() throws Exception { Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), 1234L); } - @Test + @Test(timeout = 30000) public void testMetadataSetToInRecoveryDuringFailureHandler() throws Exception { MockClientContext clientCtx = MockClientContext.create(); Versioned md = ClientUtil.setupLedger(clientCtx, 10L, @@ -350,7 +350,7 @@ public void testMetadataSetToInRecoveryDuringFailureHandler() throws Exception { Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(0L), Lists.newArrayList(b1, b2, b3)); } - @Test + @Test(timeout = 30000) public void testOldEnsembleChangedDuringFailureHandler() throws Exception { MockClientContext clientCtx = MockClientContext.create(); Versioned md = ClientUtil.setupLedger(clientCtx, 10L, @@ -403,7 +403,7 @@ public void testOldEnsembleChangedDuringFailureHandler() throws Exception { Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(2L), Lists.newArrayList(b5, b2, b4)); } - @Test + @Test(timeout = 30000) public void testNoAddsAreCompletedWhileFailureHandlingInProgress() throws Exception { MockClientContext clientCtx = MockClientContext.create(); Versioned md = ClientUtil.setupLedger(clientCtx, 10L, @@ -445,7 +445,7 @@ public void testNoAddsAreCompletedWhileFailureHandlingInProgress() throws Except Assert.assertEquals(lh.getLedgerMetadata().getAllEnsembles().get(1L), Lists.newArrayList(b1, b2, b4)); } - @Test + @Test(timeout = 30000) public void testHandleFailureBookieNotInWriteSet() throws Exception { MockClientContext clientCtx = MockClientContext.create(); Versioned md = ClientUtil.setupLedger(clientCtx, 10L, diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerHandleAdapter.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerHandleAdapter.java index 7d5cad6531a..086e9f330c5 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerHandleAdapter.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerHandleAdapter.java @@ -20,7 +20,7 @@ package org.apache.bookkeeper.client; import io.netty.buffer.ByteBuf; -import org.apache.bookkeeper.util.ByteBufList; +import org.apache.bookkeeper.proto.MockBookieClient; /** * Adapter for tests to get the public access from LedgerHandle for its default @@ -28,8 +28,9 @@ */ public class LedgerHandleAdapter { - public static ByteBufList toSend(LedgerHandle lh, long entryId, ByteBuf data) { - return lh.getDigestManager().computeDigestAndPackageForSending(entryId, lh.getLastAddConfirmed(), - lh.addToLength(data.readableBytes()), data); + public static ByteBuf toSend(LedgerHandle lh, long entryId, ByteBuf data) { + return MockBookieClient.copyData(lh.getDigestManager() + .computeDigestAndPackageForSending(entryId, lh.getLastAddConfirmed(), + lh.addToLength(data.readableBytes()), data, new byte[20], 0)); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java index 63255310c30..e7710124707 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/MockBookKeeperTestCase.java @@ -35,6 +35,7 @@ import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.Unpooled; import io.netty.buffer.UnpooledByteBufAllocator; +import io.netty.util.ReferenceCounted; import java.security.GeneralSecurityException; import java.util.ArrayList; import java.util.Collections; @@ -67,6 +68,7 @@ import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; +import org.apache.bookkeeper.proto.MockBookieClient; import org.apache.bookkeeper.proto.checksum.DigestManager; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.ByteBufList; @@ -504,10 +506,10 @@ protected void setupBookieClientReadEntry() { if (mockEntry != null) { LOG.info("readEntry - found mock entry {}@{} at {}", entryId, ledgerId, bookieSocketAddress); - ByteBufList entry = macManager.computeDigestAndPackageForSending(entryId, + ReferenceCounted entry = macManager.computeDigestAndPackageForSending(entryId, mockEntry.lastAddConfirmed, mockEntry.payload.length, - Unpooled.wrappedBuffer(mockEntry.payload)); - callback.readEntryComplete(BKException.Code.OK, ledgerId, entryId, ByteBufList.coalesce(entry), + Unpooled.wrappedBuffer(mockEntry.payload), new byte[20], 0); + callback.readEntryComplete(BKException.Code.OK, ledgerId, entryId, MockBookieClient.copyData(entry), args[4]); entry.release(); } else { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java index 0fa2f0d7762..4efc4465e38 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java @@ -26,6 +26,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import io.netty.util.ReferenceCounted; import java.io.IOException; import java.util.Enumeration; import java.util.concurrent.CompletableFuture; @@ -61,7 +62,6 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.proto.checksum.DigestManager; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; -import org.apache.bookkeeper.util.ByteBufList; import org.apache.bookkeeper.versioning.Version; import org.apache.bookkeeper.versioning.Versioned; import org.apache.commons.lang3.mutable.MutableInt; @@ -425,9 +425,10 @@ public void testRecoveryOnEntryGap() throws Exception { long entryId = 14; long lac = 8; byte[] data = "recovery-on-entry-gap-gap".getBytes(UTF_8); - ByteBufList toSend = + ReferenceCounted toSend = lh.macManager.computeDigestAndPackageForSending( - entryId, lac, lh.getLength() + 100, Unpooled.wrappedBuffer(data, 0, data.length)); + entryId, lac, lh.getLength() + 100, Unpooled.wrappedBuffer(data, 0, data.length), + new byte[20], 0); final CountDownLatch addLatch = new CountDownLatch(1); final AtomicBoolean addSuccess = new AtomicBoolean(false); LOG.info("Add entry {} with lac = {}", entryId, lac); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java index b538a50c2a0..8e3cfd72e42 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOpTest.java @@ -33,6 +33,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.buffer.UnpooledByteBufAllocator; +import io.netty.util.ReferenceCounted; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -166,10 +167,15 @@ public void testSpeculativeResponses() throws Exception { final long lac = 1L; ByteBuf data = Unpooled.copiedBuffer("test-speculative-responses", UTF_8); - ByteBufList dataWithDigest = digestManager.computeDigestAndPackageForSending( - entryId, lac, data.readableBytes(), data); - byte[] bytesWithDigest = new byte[dataWithDigest.readableBytes()]; - assertEquals(bytesWithDigest.length, dataWithDigest.getBytes(bytesWithDigest)); + ReferenceCounted refCnt = digestManager.computeDigestAndPackageForSending( + entryId, lac, data.readableBytes(), data, new byte[20], 0); + + byte[] bytesWithDigest = null; + if (refCnt instanceof ByteBufList) { + ByteBufList dataWithDigest = (ByteBufList) refCnt; + bytesWithDigest = new byte[dataWithDigest.readableBytes()]; + assertEquals(bytesWithDigest.length, dataWithDigest.getBytes(bytesWithDigest)); + } final Map callbacks = Collections.synchronizedMap(new HashMap<>()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestPendingReadLacOp.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestPendingReadLacOp.java index 82b031c5ae8..a37462dee7d 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestPendingReadLacOp.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestPendingReadLacOp.java @@ -23,8 +23,10 @@ import static org.junit.Assert.assertEquals; import io.netty.buffer.Unpooled; +import io.netty.util.ReferenceCounted; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import org.apache.bookkeeper.proto.MockBookieClient; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.util.ByteBufList; import org.junit.Test; @@ -57,17 +59,20 @@ public void testPendingReadLacOpMissingExplicitLAC() throws Exception { public void initiate() { for (int i = 0; i < lh.getCurrentEnsemble().size(); i++) { final int index = i; - ByteBufList buffer = lh.getDigestManager().computeDigestAndPackageForSending( + ReferenceCounted toSend = lh.getDigestManager().computeDigestAndPackageForSending( 2, 1, data.length, - Unpooled.wrappedBuffer(data)); + Unpooled.wrappedBuffer(data), + new byte[20], + 0); + bkc.scheduler.schedule(() -> { readLacComplete( 0, lh.getId(), null, - Unpooled.copiedBuffer(buffer.toArray()), + MockBookieClient.copyData(toSend), index); }, 0, TimeUnit.SECONDS); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/AbstractConfigurationTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/AbstractConfigurationTest.java index 54e07bd5992..a6333a47d32 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/AbstractConfigurationTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/conf/AbstractConfigurationTest.java @@ -22,6 +22,7 @@ import static org.mockito.Mockito.CALLS_REAL_METHODS; import static org.mockito.Mockito.mock; +import org.apache.bookkeeper.common.allocator.LeakDetectionPolicy; import org.apache.bookkeeper.meta.AbstractZkLedgerManagerFactory; import org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerManagerFactory; @@ -130,4 +131,52 @@ public void testUnknownZkLedgerManagerFactory() throws Exception { conf.getMetadataServiceUri(); } + @Test + public void testAllocatorLeakDetectionPolicy() { + String nettyOldLevelKey = "io.netty.leakDetectionLevel"; + String nettyLevelKey = "io.netty.leakDetection.level"; + + String nettyOldLevelStr = System.getProperty(nettyOldLevelKey); + String nettyLevelStr = System.getProperty(nettyLevelKey); + + //Remove netty property for test. + System.getProperties().remove(nettyOldLevelKey); + System.getProperties().remove(nettyLevelKey); + + assertEquals(LeakDetectionPolicy.Disabled, conf.getAllocatorLeakDetectionPolicy()); + + System.getProperties().put(nettyOldLevelKey, "zazaza"); + assertEquals(LeakDetectionPolicy.Disabled, conf.getAllocatorLeakDetectionPolicy()); + + conf.setProperty(AbstractConfiguration.ALLOCATOR_LEAK_DETECTION_POLICY, "zazaza"); + assertEquals(LeakDetectionPolicy.Disabled, conf.getAllocatorLeakDetectionPolicy()); + + System.getProperties().put(nettyOldLevelKey, "simple"); + assertEquals(LeakDetectionPolicy.Simple, conf.getAllocatorLeakDetectionPolicy()); + + System.getProperties().put(nettyLevelKey, "disabled"); + assertEquals(LeakDetectionPolicy.Disabled, conf.getAllocatorLeakDetectionPolicy()); + + System.getProperties().put(nettyLevelKey, "advanCed"); + assertEquals(LeakDetectionPolicy.Advanced, conf.getAllocatorLeakDetectionPolicy()); + + conf.setProperty(AbstractConfiguration.ALLOCATOR_LEAK_DETECTION_POLICY, "simPle"); + assertEquals(LeakDetectionPolicy.Advanced, conf.getAllocatorLeakDetectionPolicy()); + + conf.setProperty(AbstractConfiguration.ALLOCATOR_LEAK_DETECTION_POLICY, "advanCed"); + assertEquals(LeakDetectionPolicy.Advanced, conf.getAllocatorLeakDetectionPolicy()); + + conf.setProperty(AbstractConfiguration.ALLOCATOR_LEAK_DETECTION_POLICY, "paranoiD"); + assertEquals(LeakDetectionPolicy.Paranoid, conf.getAllocatorLeakDetectionPolicy()); + + System.getProperties().remove(nettyOldLevelKey); + System.getProperties().remove(nettyLevelKey); + //Revert the netty properties. + if (nettyOldLevelStr != null) { + System.getProperties().put(nettyOldLevelKey, nettyOldLevelStr); + } + if (nettyLevelStr != null) { + System.getProperties().put(nettyLevelKey, nettyLevelStr); + } + } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieBackpressureForV2Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieBackpressureForV2Test.java index ce8d65fb76b..8721a2c7819 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieBackpressureForV2Test.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/BookieBackpressureForV2Test.java @@ -18,6 +18,8 @@ */ package org.apache.bookkeeper.proto; +import org.apache.bookkeeper.client.BookKeeperTestClient; +import org.apache.bookkeeper.test.TestStatsProvider; import org.junit.Before; /** @@ -30,6 +32,8 @@ public class BookieBackpressureForV2Test extends BookieBackpressureTest { public void setUp() throws Exception { super.setUp(); baseClientConf.setUseV2WireProtocol(true); + bkc = new BookKeeperTestClient(baseClientConf, new TestStatsProvider()); + // the backpressure will bloc the read response, disable it to let it use backpressure mechanism confByIndex(0).setReadWorkerThreadsThrottlingEnabled(false); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java index 2de9ab5e19a..c4344c74d00 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/MockBookieClient.java @@ -24,6 +24,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import io.netty.util.ReferenceCounted; import java.util.Arrays; import java.util.Collections; import java.util.EnumSet; @@ -148,7 +149,7 @@ public void writeLac(BookieId addr, long ledgerId, byte[] masterKey, @Override public void addEntry(BookieId addr, long ledgerId, byte[] masterKey, - long entryId, ByteBufList toSend, WriteCallback cb, Object ctx, + long entryId, ReferenceCounted toSend, WriteCallback cb, Object ctx, int options, boolean allowFastFail, EnumSet writeFlags) { toSend.retain(); preWriteHook.runHook(addr, ledgerId, entryId) @@ -262,11 +263,29 @@ public boolean isClosed() { public void close() { } - private static ByteBuf copyData(ByteBufList list) { - ByteBuf buf = Unpooled.buffer(list.readableBytes()); - for (int i = 0; i < list.size(); i++) { - buf.writeBytes(list.getBuffer(i).slice()); + public static ByteBuf copyData(ReferenceCounted rc) { + ByteBuf res; + if (rc instanceof ByteBuf) { + res = Unpooled.copiedBuffer((ByteBuf) rc); + } else { + res = ByteBufList.coalesce((ByteBufList) rc); } - return buf; + + return res; + } + + public static ByteBuf copyDataWithSkipHeader(ReferenceCounted rc) { + ByteBuf res; + if (rc instanceof ByteBuf) { + res = Unpooled.copiedBuffer((ByteBuf) rc); + } else { + res = ByteBufList.coalesce((ByteBufList) rc); + } + + // Skip headers + res.skipBytes(28); + rc.release(); + + return res; } -} \ No newline at end of file +} 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 c670e87ff0c..cef77c3f99a 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 @@ -84,8 +84,8 @@ public ByteBuf generateEntry(long ledgerId, long entryId, long lac) throws Excep DigestManager digestManager = DigestManager.instantiate(ledgerId, new byte[0], DataFormats.LedgerMetadataFormat.DigestType.CRC32C, UnpooledByteBufAllocator.DEFAULT, false); - return ByteBufList.coalesce(digestManager.computeDigestAndPackageForSending( - entryId, lac, 0, Unpooled.buffer(10))); + 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/proto/ReadEntryProcessorTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ReadEntryProcessorTest.java index 91e100d8093..251f900c096 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ReadEntryProcessorTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/ReadEntryProcessorTest.java @@ -107,7 +107,7 @@ private void testAsynchronousRequest(boolean result, int errorCode) throws Excep ExecutorService service = Executors.newCachedThreadPool(); long ledgerId = System.currentTimeMillis(); - ReadRequest request = new ReadRequest(BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, + ReadRequest request = ReadRequest.create(BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, 1, BookieProtocol.FLAG_DO_FENCING, new byte[]{}); ReadEntryProcessor processor = ReadEntryProcessor.create( request, requestHandler, requestProcessor, service, true); @@ -150,7 +150,7 @@ private void testSynchronousRequest(boolean result, int errorCode) throws Except }).when(channel).writeAndFlush(any(Response.class)); long ledgerId = System.currentTimeMillis(); - ReadRequest request = new ReadRequest(BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, + ReadRequest request = ReadRequest.create(BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, 1, BookieProtocol.FLAG_DO_FENCING, new byte[]{}); ReadEntryProcessor processor = ReadEntryProcessor.create(request, requestHandler, requestProcessor, null, true); fenceResult.complete(result); @@ -180,7 +180,7 @@ public void testNonFenceRequest() throws Exception { }).when(channel).writeAndFlush(any(Response.class)); long ledgerId = System.currentTimeMillis(); - ReadRequest request = new ReadRequest(BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, + ReadRequest request = ReadRequest.create(BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, 1, (short) 0, new byte[]{}); ReadEntryProcessor processor = ReadEntryProcessor.create(request, requestHandler, requestProcessor, null, true); processor.run(); 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 23997348e1b..1edd74c1fc6 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 @@ -161,8 +161,9 @@ public void testAuthFail() throws Exception { } - client.sendRequest(new ReadRequest(BookieProtocol.CURRENT_PROTOCOL_VERSION, - 1L, 1L, (short) 0, null)); + ReadRequest read = ReadRequest.create(BookieProtocol.CURRENT_PROTOCOL_VERSION, + 1L, 1L, (short) 0, null); + client.sendRequest(read); Response response = client.takeResponse(); assertEquals("Should have failed", response.getErrorCode(), BookieProtocol.EUA); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorCheckAllLedgersTaskTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorCheckAllLedgersTaskTest.java index 320f767282d..02f585306f5 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorCheckAllLedgersTaskTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorCheckAllLedgersTaskTest.java @@ -90,7 +90,7 @@ public void testCheckAllLedgers() throws Exception { AuditorCheckAllLedgersTask auditorCheckAllLedgersTask = new AuditorCheckAllLedgersTask( baseConf, auditorStats, admin, ledgerManager, - ledgerUnderreplicationManager, null); + ledgerUnderreplicationManager, null, (flag, throwable) -> flag.set(false)); // 3. checkAllLedgers auditorCheckAllLedgersTask.runTask(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java index 41095a97603..9257419c3a0 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java @@ -41,6 +41,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -72,6 +73,7 @@ import org.apache.bookkeeper.test.TestStatsProvider; import org.apache.bookkeeper.test.TestStatsProvider.TestOpStatsLogger; import org.apache.bookkeeper.test.TestStatsProvider.TestStatsLogger; +import org.awaitility.Awaitility; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -445,7 +447,7 @@ void validateInitialDelayOfCheckAllLedgers(LedgerUnderreplicationManager urm, lo servConf.setAuditorPeriodicBookieCheckInterval(0); final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, bkc, false, - statsLogger); + statsLogger, null); CountDownLatch latch = auditor.getLatch(); assertEquals("CHECK_ALL_LEDGERS_TIME SuccessCount", 0, checkAllLedgersStatsLogger.getSuccessCount()); long curTimeBeforeStart = System.currentTimeMillis(); @@ -539,7 +541,7 @@ void validateInitialDelayOfPlacementPolicyCheck(LedgerUnderreplicationManager ur servConf.setAuditorPeriodicBookieCheckInterval(0); final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, bkc, false, - statsLogger); + statsLogger, null); CountDownLatch latch = auditor.getLatch(); assertEquals("PLACEMENT_POLICY_CHECK_TIME SuccessCount", 0, placementPolicyCheckStatsLogger.getSuccessCount()); long curTimeBeforeStart = System.currentTimeMillis(); @@ -643,7 +645,7 @@ void validateInitialDelayOfReplicasCheck(LedgerUnderreplicationManager urm, long servConf.setAuditorPeriodicCheckInterval(0); servConf.setAuditorPeriodicBookieCheckInterval(0); final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, bkc, false, - statsLogger); + statsLogger, null); CountDownLatch latch = auditor.getLatch(); assertEquals("REPLICAS_CHECK_TIME SuccessCount", 0, replicasCheckStatsLogger.getSuccessCount()); long curTimeBeforeStart = System.currentTimeMillis(); @@ -699,35 +701,222 @@ void validateInitialDelayOfReplicasCheck(LedgerUnderreplicationManager urm, long auditor.close(); } + @Test + public void testDelayBookieAuditOfCheckAllLedgers() throws Exception { + for (AuditorElector e : auditorElectors.values()) { + e.shutdown(); + } + + final int numLedgers = 10; + List ids = new LinkedList(); + for (int i = 0; i < numLedgers; i++) { + LedgerHandle lh = bkc.createLedger(3, 3, DigestType.CRC32, "passwd".getBytes()); + ids.add(lh.getId()); + for (int j = 0; j < 2; j++) { + lh.addEntry("testdata".getBytes()); + } + lh.close(); + } + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerUnderreplicationManager urm = mFactory.newLedgerUnderreplicationManager(); + + ServerConfiguration servConf = new ServerConfiguration(confByIndex(0)); + + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + Counter numBookieAuditsDelayed = + statsLogger.getCounter(ReplicationStats.NUM_BOOKIE_AUDITS_DELAYED); + TestOpStatsLogger underReplicatedLedgerTotalSizeStatsLogger = (TestOpStatsLogger) statsLogger + .getOpStatsLogger(ReplicationStats.UNDER_REPLICATED_LEDGERS_TOTAL_SIZE); + + servConf.setAuditorPeriodicCheckInterval(1); + servConf.setAuditorPeriodicPlacementPolicyCheckInterval(0); + servConf.setAuditorPeriodicBookieCheckInterval(Long.MAX_VALUE); + + urm.setLostBookieRecoveryDelay(Integer.MAX_VALUE); + + AtomicBoolean canRun = new AtomicBoolean(false); + + final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, bkc, + false, statsLogger, canRun); + final CountDownLatch latch = auditor.getLatch(); + + auditor.start(); + + killBookie(addressByIndex(0)); + + Awaitility.await().untilAsserted(() -> assertEquals(1, (long) numBookieAuditsDelayed.get())); + final Future auditTask = auditor.auditTask; + assertTrue(auditTask != null && !auditTask.isDone()); + + canRun.set(true); + + assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertTrue(auditor.auditTask.equals(auditTask) + && auditor.auditTask != null && !auditor.auditTask.isDone()); + // wrong num is numLedgers, right num is 0 + assertEquals("UNDER_REPLICATED_LEDGERS_TOTAL_SIZE", + 0, + underReplicatedLedgerTotalSizeStatsLogger.getSuccessCount()); + + auditor.close(); + } + + @Test + public void testDelayBookieAuditOfPlacementPolicy() throws Exception { + for (AuditorElector e : auditorElectors.values()) { + e.shutdown(); + } + + final int numLedgers = 10; + List ids = new LinkedList(); + for (int i = 0; i < numLedgers; i++) { + LedgerHandle lh = bkc.createLedger(3, 3, DigestType.CRC32, "passwd".getBytes()); + ids.add(lh.getId()); + for (int j = 0; j < 2; j++) { + lh.addEntry("testdata".getBytes()); + } + lh.close(); + } + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerUnderreplicationManager urm = mFactory.newLedgerUnderreplicationManager(); + + ServerConfiguration servConf = new ServerConfiguration(confByIndex(0)); + + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + Counter numBookieAuditsDelayed = + statsLogger.getCounter(ReplicationStats.NUM_BOOKIE_AUDITS_DELAYED); + TestOpStatsLogger placementPolicyCheckTime = (TestOpStatsLogger) statsLogger + .getOpStatsLogger(ReplicationStats.PLACEMENT_POLICY_CHECK_TIME); + + servConf.setAuditorPeriodicCheckInterval(0); + servConf.setAuditorPeriodicPlacementPolicyCheckInterval(1); + servConf.setAuditorPeriodicBookieCheckInterval(Long.MAX_VALUE); + + urm.setLostBookieRecoveryDelay(Integer.MAX_VALUE); + + AtomicBoolean canRun = new AtomicBoolean(false); + + final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, bkc, + false, statsLogger, canRun); + final CountDownLatch latch = auditor.getLatch(); + + auditor.start(); + + killBookie(addressByIndex(0)); + + Awaitility.await().untilAsserted(() -> assertEquals(1, (long) numBookieAuditsDelayed.get())); + final Future auditTask = auditor.auditTask; + assertTrue(auditTask != null && !auditTask.isDone()); + assertEquals("PLACEMENT_POLICY_CHECK_TIME", 0, placementPolicyCheckTime.getSuccessCount()); + + canRun.set(true); + + assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertTrue(auditor.auditTask.equals(auditTask) + && auditor.auditTask != null && !auditor.auditTask.isDone()); + // wrong successCount is > 0, right successCount is = 0 + assertEquals("PLACEMENT_POLICY_CHECK_TIME", 0, placementPolicyCheckTime.getSuccessCount()); + + auditor.close(); + } + + @Test + public void testDelayBookieAuditOfReplicasCheck() throws Exception { + for (AuditorElector e : auditorElectors.values()) { + e.shutdown(); + } + + final int numLedgers = 10; + List ids = new LinkedList(); + for (int i = 0; i < numLedgers; i++) { + LedgerHandle lh = bkc.createLedger(3, 3, DigestType.CRC32, "passwd".getBytes()); + ids.add(lh.getId()); + for (int j = 0; j < 2; j++) { + lh.addEntry("testdata".getBytes()); + } + lh.close(); + } + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerUnderreplicationManager urm = mFactory.newLedgerUnderreplicationManager(); + + ServerConfiguration servConf = new ServerConfiguration(confByIndex(0)); + + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + Counter numBookieAuditsDelayed = + statsLogger.getCounter(ReplicationStats.NUM_BOOKIE_AUDITS_DELAYED); + TestOpStatsLogger replicasCheckTime = (TestOpStatsLogger) statsLogger + .getOpStatsLogger(ReplicationStats.REPLICAS_CHECK_TIME); + + servConf.setAuditorPeriodicCheckInterval(0); + servConf.setAuditorPeriodicPlacementPolicyCheckInterval(0); + servConf.setAuditorPeriodicBookieCheckInterval(Long.MAX_VALUE); + servConf.setAuditorPeriodicReplicasCheckInterval(1); + + urm.setLostBookieRecoveryDelay(Integer.MAX_VALUE); + + AtomicBoolean canRun = new AtomicBoolean(false); + + final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, bkc, + false, statsLogger, canRun); + final CountDownLatch latch = auditor.getLatch(); + + auditor.start(); + + killBookie(addressByIndex(0)); + + Awaitility.await().untilAsserted(() -> assertEquals(1, (long) numBookieAuditsDelayed.get())); + final Future auditTask = auditor.auditTask; + assertTrue(auditTask != null && !auditTask.isDone()); + assertEquals("REPLICAS_CHECK_TIME", 0, replicasCheckTime.getSuccessCount()); + + canRun.set(true); + + assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertTrue(auditor.auditTask.equals(auditTask) + && auditor.auditTask != null && !auditor.auditTask.isDone()); + // wrong successCount is > 0, right successCount is = 0 + assertEquals("REPLICAS_CHECK_TIME", 0, replicasCheckTime.getSuccessCount()); + + auditor.close(); + } + static class TestAuditor extends Auditor { final AtomicReference latchRef = new AtomicReference(new CountDownLatch(1)); public TestAuditor(String bookieIdentifier, ServerConfiguration conf, BookKeeper bkc, boolean ownBkc, - StatsLogger statsLogger) throws UnavailableException { + StatsLogger statsLogger, AtomicBoolean exceptedRun) throws UnavailableException { super(bookieIdentifier, conf, bkc, ownBkc, statsLogger); - renewAuditorTestWrapperTask(); + renewAuditorTestWrapperTask(exceptedRun); } public TestAuditor(String bookieIdentifier, ServerConfiguration conf, BookKeeper bkc, boolean ownBkc, - BookKeeperAdmin bkadmin, boolean ownadmin, StatsLogger statsLogger) throws UnavailableException { + BookKeeperAdmin bkadmin, boolean ownadmin, StatsLogger statsLogger, + AtomicBoolean exceptedRun) throws UnavailableException { super(bookieIdentifier, conf, bkc, ownBkc, bkadmin, ownadmin, statsLogger); - renewAuditorTestWrapperTask(); + renewAuditorTestWrapperTask(exceptedRun); } - public TestAuditor(final String bookieIdentifier, ServerConfiguration conf, StatsLogger statsLogger) + public TestAuditor(final String bookieIdentifier, ServerConfiguration conf, StatsLogger statsLogger, + AtomicBoolean exceptedRun) throws UnavailableException { super(bookieIdentifier, conf, statsLogger); - renewAuditorTestWrapperTask(); + renewAuditorTestWrapperTask(exceptedRun); } - private void renewAuditorTestWrapperTask() { + private void renewAuditorTestWrapperTask(AtomicBoolean exceptedRun) { super.auditorCheckAllLedgersTask = - new AuditorTestWrapperTask(super.auditorCheckAllLedgersTask, latchRef); + new AuditorTestWrapperTask(super.auditorCheckAllLedgersTask, latchRef, exceptedRun); super.auditorPlacementPolicyCheckTask = - new AuditorTestWrapperTask(super.auditorPlacementPolicyCheckTask, latchRef); + new AuditorTestWrapperTask(super.auditorPlacementPolicyCheckTask, latchRef, exceptedRun); super.auditorReplicasCheckTask = - new AuditorTestWrapperTask(super.auditorReplicasCheckTask, latchRef); + new AuditorTestWrapperTask(super.auditorReplicasCheckTask, latchRef, exceptedRun); } CountDownLatch getLatch() { @@ -741,18 +930,24 @@ void setLatch(CountDownLatch latch) { private static class AuditorTestWrapperTask extends AuditorTask { private final AuditorTask innerTask; private final AtomicReference latchRef; + private final AtomicBoolean exceptedRun; - AuditorTestWrapperTask(AuditorTask innerTask, AtomicReference latchRef) { + AuditorTestWrapperTask(AuditorTask innerTask, + AtomicReference latchRef, + AtomicBoolean exceptedRun) { super(null, null, null, null, null, - null); + null, null); this.innerTask = innerTask; this.latchRef = latchRef; + this.exceptedRun = exceptedRun; } @Override protected void runTask() { - innerTask.runTask(); - latchRef.get().countDown(); + if (exceptedRun == null || exceptedRun.get()) { + innerTask.runTask(); + latchRef.get().countDown(); + } } @Override diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTaskTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTaskTest.java index ec773efcf64..6d951d71803 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTaskTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTaskTest.java @@ -86,7 +86,7 @@ public void testPlacementPolicyCheck() throws BKException, InterruptedException AuditorPlacementPolicyCheckTask auditorPlacementPolicyCheckTask = new AuditorPlacementPolicyCheckTask( baseConf, auditorStats, admin, ledgerManager, - ledgerUnderreplicationManager, null); + ledgerUnderreplicationManager, null, (flag, throwable) -> flag.set(false)); // 3. placementPolicyCheck auditorPlacementPolicyCheckTask.runTask(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java index b26084921d3..e14deb3fb59 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java @@ -831,7 +831,7 @@ private TestStatsLogger startAuditorAndWaitForPlacementPolicyCheck(ServerConfigu .getOpStatsLogger(ReplicationStats.PLACEMENT_POLICY_CHECK_TIME); final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, - statsLogger); + statsLogger, null); auditorRef.setValue(auditor); CountDownLatch latch = auditor.getLatch(); assertEquals("PLACEMENT_POLICY_CHECK_TIME SuccessCount", 0, placementPolicyCheckStatsLogger.getSuccessCount()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTaskTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTaskTest.java index 34c440b06ca..f39a9dace7f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTaskTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTaskTest.java @@ -85,7 +85,7 @@ public void testReplicasCheck() throws BKException, InterruptedException { final AuditorStats auditorStats = new AuditorStats(statsLogger); AuditorReplicasCheckTask auditorReplicasCheckTask = new AuditorReplicasCheckTask( baseConf, auditorStats, admin, ledgerManager, - ledgerUnderreplicationManager, null); + ledgerUnderreplicationManager, null, (flag, throwable) -> flag.set(false)); // 3. replicasCheck auditorReplicasCheckTask.runTask(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java index a9377f394e4..c64a14eca28 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java @@ -155,7 +155,7 @@ private TestStatsLogger startAuditorAndWaitForReplicasCheck(ServerConfiguration final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, bkc, true, new TestBookKeeperAdmin(bkc, statsLogger, expectedReturnAvailabilityOfEntriesOfLedger, errorReturnValueForGetAvailabilityOfEntriesOfLedger), - true, statsLogger); + true, statsLogger, null); auditorRef.setValue(auditor); CountDownLatch latch = auditor.getLatch(); assertEquals("REPLICAS_CHECK_TIME SuccessCount", 0, replicasCheckStatsLogger.getSuccessCount()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java index b7752b8bcbf..fab068ae609 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java @@ -1374,7 +1374,7 @@ private TestStatsLogger startAuditorAndWaitForPlacementPolicyCheck(ServerConfigu .getOpStatsLogger(ReplicationStats.PLACEMENT_POLICY_CHECK_TIME); final AuditorPeriodicCheckTest.TestAuditor auditor = new AuditorPeriodicCheckTest.TestAuditor( - BookieImpl.getBookieId(servConf).toString(), servConf, bkc, false, statsLogger); + BookieImpl.getBookieId(servConf).toString(), servConf, bkc, false, statsLogger, null); auditorRef.setValue(auditor); CountDownLatch latch = auditor.getLatch(); assertEquals("PLACEMENT_POLICY_CHECK_TIME SuccessCount", 0, diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java index 5fb1679810a..fdab2f8f9ad 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/server/http/TestHttpService.java @@ -23,17 +23,22 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.when; +import static org.powermock.api.mockito.PowerMockito.spy; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.util.concurrent.UncheckedExecutionException; import java.io.File; +import java.lang.reflect.Field; import java.util.HashMap; import java.util.LinkedHashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.Future; import lombok.Cleanup; import org.apache.bookkeeper.bookie.BookieResources; +import org.apache.bookkeeper.bookie.LedgerStorage; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.ClientUtil; import org.apache.bookkeeper.client.LedgerHandle; @@ -50,6 +55,7 @@ import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.meta.MetadataBookieDriver; import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.replication.AuditorElector; import org.apache.bookkeeper.server.http.service.BookieInfoService; import org.apache.bookkeeper.server.http.service.BookieSanityService; @@ -1087,4 +1093,79 @@ public void testSuspendCompaction() throws Exception { assertEquals(responseMap7.get("isMajorGcSuspended"), "false"); assertEquals(responseMap7.get("isMinorGcSuspended"), "false"); } + + @Test + public void testTriggerEntryLocationCompactService() throws Exception { + BookieServer bookieServer = serverByIndex(numberOfBookies - 1); + LedgerStorage spyLedgerStorage = spy(bookieServer.getBookie().getLedgerStorage()); + List dbLocationPath = Lists.newArrayList("/data1/bookkeeper/ledgers/current/locations", + "/data2/bookkeeper/ledgers/current/locations"); + when(spyLedgerStorage.getEntryLocationDBPath()) + .thenReturn(dbLocationPath); + + HashMap statusMap = Maps.newHashMap(); + statusMap.put("/data1/bookkeeper/ledgers/current/locations", false); + statusMap.put("/data2/bookkeeper/ledgers/current/locations", true); + when(spyLedgerStorage.isEntryLocationCompacting(dbLocationPath)) + .thenReturn(statusMap); + + Field ledgerStorageField = bookieServer.getBookie().getClass().getDeclaredField("ledgerStorage"); + ledgerStorageField.setAccessible(true); + ledgerStorageField.set(bookieServer.getBookie(), spyLedgerStorage); + + HttpEndpointService triggerEntryLocationCompactService = bkHttpServiceProvider + .provideHttpEndpointService(HttpServer.ApiType.TRIGGER_ENTRY_LOCATION_COMPACT); + + // 1. Put + // 1.1 Trigger all entry location rocksDB compact, should return OK + HttpServiceRequest request1 = new HttpServiceRequest("{\"entryLocationRocksDBCompact\":true}", + HttpServer.Method.PUT, null); + HttpServiceResponse response1 = triggerEntryLocationCompactService.handle(request1); + assertEquals(HttpServer.StatusCode.OK.getValue(), response1.getStatusCode()); + LOG.info("Get response: {}", response1.getBody()); + + // 1.2 Specified trigger entry location rocksDB compact, should return OK + String body2 = "{\"entryLocationRocksDBCompact\":true,\"entryLocations\"" + + ":\"/data1/bookkeeper/ledgers/current/locations\"}"; + HttpServiceRequest request2 = new HttpServiceRequest(body2, HttpServer.Method.PUT, null); + HttpServiceResponse response2 = triggerEntryLocationCompactService.handle(request2); + assertEquals(HttpServer.StatusCode.OK.getValue(), response2.getStatusCode()); + LOG.info("Get response: {}", response2.getBody()); + assertTrue(response2.getBody().contains("Triggered entry Location RocksDB")); + + // 1.3 Specified invalid entry location rocksDB compact, should return BAD_REQUEST + String body3 = "{\"entryLocationRocksDBCompact\":true,\"entryLocations\"" + + ":\"/invalid1/locations,/data2/bookkeeper/ledgers/current/locations\"}"; + HttpServiceRequest request3 = new HttpServiceRequest(body3, HttpServer.Method.PUT, null); + HttpServiceResponse response3 = triggerEntryLocationCompactService.handle(request3); + assertEquals(HttpServer.StatusCode.BAD_REQUEST.getValue(), response3.getStatusCode()); + LOG.info("Get response: {}", response3.getBody()); + assertTrue(response3.getBody().contains("is invalid")); + + // 1.4 Some rocksDB is running compact, should return OK + String body4 = "{\"entryLocationRocksDBCompact\":true,\"entryLocations\"" + + ":\"/data1/bookkeeper/ledgers/current/locations,/data2/bookkeeper/ledgers/current/locations\"}"; + HttpServiceRequest request4 = new HttpServiceRequest(body4, HttpServer.Method.PUT, null); + HttpServiceResponse response4 = triggerEntryLocationCompactService.handle(request4); + assertEquals(HttpServer.StatusCode.OK.getValue(), response4.getStatusCode()); + LOG.info("Get response: {}", response4.getBody()); + + // 1.5 Put, empty body, should return BAD_REQUEST + HttpServiceRequest request5 = new HttpServiceRequest(null, HttpServer.Method.PUT, null); + HttpServiceResponse response5 = triggerEntryLocationCompactService.handle(request5); + assertEquals(HttpServer.StatusCode.BAD_REQUEST.getValue(), response5.getStatusCode()); + LOG.info("Get response: {}", response5.getBody()); + + // 2. GET, should return OK + HttpServiceRequest request6 = new HttpServiceRequest(null, HttpServer.Method.GET, null); + HttpServiceResponse response6 = triggerEntryLocationCompactService.handle(request6); + assertEquals(HttpServer.StatusCode.OK.getValue(), response6.getStatusCode()); + assertTrue(response6.getBody().contains("\"/data2/bookkeeper/ledgers/current/locations\" : true")); + assertTrue(response6.getBody().contains("\"/data1/bookkeeper/ledgers/current/locations\" : false")); + + // 3. POST, should return NOT_FOUND + HttpServiceRequest request7 = new HttpServiceRequest(null, HttpServer.Method.POST, null); + HttpServiceResponse response7 = triggerEntryLocationCompactService.handle(request7); + assertEquals(HttpServer.StatusCode.METHOD_NOT_ALLOWED.getValue(), response7.getStatusCode()); + } } diff --git a/bookkeeper-server/src/test/resources/conf/entry_location_rocksdb.conf b/bookkeeper-server/src/test/resources/conf/entry_location_rocksdb.conf index df3ac9fc2ec..6f6c1b4d052 100644 --- a/bookkeeper-server/src/test/resources/conf/entry_location_rocksdb.conf +++ b/bookkeeper-server/src/test/resources/conf/entry_location_rocksdb.conf @@ -51,6 +51,8 @@ max_bytes_for_level_base=268435456 # set by jni: options.setTargetFileSizeBase target_file_size_base=67108864 + # set by jni: options.setLevelCompactionDynamicLevelBytes + level_compaction_dynamic_level_bytes=true [TableOptions/BlockBasedTable "default"] # set by jni: tableOptions.setBlockSize @@ -64,6 +66,4 @@ # set by jni: tableOptions.setFilterPolicy, bloomfilter:[bits_per_key]:[use_block_based_builder] filter_policy=rocksdb.BloomFilter:10:false # set by jni: tableOptions.setCacheIndexAndFilterBlocks - cache_index_and_filter_blocks=true - # set by jni: options.setLevelCompactionDynamicLevelBytes - level_compaction_dynamic_level_bytes=true \ No newline at end of file + cache_index_and_filter_blocks=true \ No newline at end of file diff --git a/bookkeeper-server/src/test/resources/test_entry_location_rocksdb.conf b/bookkeeper-server/src/test/resources/test_entry_location_rocksdb.conf index fa3cf9c8d90..9d1c3e08c47 100644 --- a/bookkeeper-server/src/test/resources/test_entry_location_rocksdb.conf +++ b/bookkeeper-server/src/test/resources/test_entry_location_rocksdb.conf @@ -31,6 +31,8 @@ write_buffer_size=1024 # set by jni: options.setMaxWriteBufferNumber max_write_buffer_number=1 + # set by jni: options.setLevelCompactionDynamicLevelBytes + level_compaction_dynamic_level_bytes=true [TableOptions/BlockBasedTable "default"] # set by jni: tableOptions.setBlockSize @@ -44,6 +46,4 @@ # set by jni: tableOptions.setFilterPolicy, bloomfilter:[bits_per_key]:[use_block_based_builder] filter_policy=rocksdb.BloomFilter:10:false # set by jni: tableOptions.setCacheIndexAndFilterBlocks - cache_index_and_filter_blocks=true - # set by jni: options.setLevelCompactionDynamicLevelBytes - level_compaction_dynamic_level_bytes=true \ No newline at end of file + cache_index_and_filter_blocks=true \ No newline at end of file diff --git a/conf/bk_server.conf b/conf/bk_server.conf index cef24eaad15..812c5b072d0 100755 --- a/conf/bk_server.conf +++ b/conf/bk_server.conf @@ -621,6 +621,15 @@ gcEntryLogMetadataCacheEnabled=false # name "entrylogIndexCache"] # gcEntryLogMetadataCachePath= +# When judging whether an entry log file need to be compacted, we calculate the usage rate of the entry log file based +# on the actual size of the entry log file. However, if an entry log file is 1MB in size and 0.9MB of data is +# being used, this entry log file won't be compacted by garbage collector due to the high usage ratio, +# which will result in many small entry log files. +# We introduced the parameter `useTargetEntryLogSizeForGc` to determine whether to calculate entry log file usage +# based on the configured target entry log file size, which is configured by `logSizeLimit`. +# Default: useTargetEntryLogSizeForGc is false. +# useTargetEntryLogSizeForGc=false + ############################################################################# ## Disk utilization ############################################################################# diff --git a/conf/entry_location_rocksdb.conf.default b/conf/entry_location_rocksdb.conf.default index df3ac9fc2ec..6f6c1b4d052 100644 --- a/conf/entry_location_rocksdb.conf.default +++ b/conf/entry_location_rocksdb.conf.default @@ -51,6 +51,8 @@ max_bytes_for_level_base=268435456 # set by jni: options.setTargetFileSizeBase target_file_size_base=67108864 + # set by jni: options.setLevelCompactionDynamicLevelBytes + level_compaction_dynamic_level_bytes=true [TableOptions/BlockBasedTable "default"] # set by jni: tableOptions.setBlockSize @@ -64,6 +66,4 @@ # set by jni: tableOptions.setFilterPolicy, bloomfilter:[bits_per_key]:[use_block_based_builder] filter_policy=rocksdb.BloomFilter:10:false # set by jni: tableOptions.setCacheIndexAndFilterBlocks - cache_index_and_filter_blocks=true - # set by jni: options.setLevelCompactionDynamicLevelBytes - level_compaction_dynamic_level_bytes=true \ No newline at end of file + cache_index_and_filter_blocks=true \ No newline at end of file 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 7eb8fa97746..308463608b5 100644 --- a/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/ProtocolBenchmark.java +++ b/microbenchmarks/src/main/java/org/apache/bookkeeper/proto/ProtocolBenchmark.java @@ -79,22 +79,6 @@ public void prepare() { this.reqEnDeV3 = new RequestEnDecoderV3(null); } - - @Benchmark - public void testAddEntryV2() throws Exception { - ByteBufList list = ByteBufList.get(entry.retainedSlice()); - BookieProtocol.AddRequest req = BookieProtocol.AddRequest.create( - BookieProtocol.CURRENT_PROTOCOL_VERSION, - ledgerId, - entryId, - flags, - masterKey, - list); - Object res = this.reqEnDeV2.encode(req, ByteBufAllocator.DEFAULT); - ReferenceCountUtil.release(res); - ReferenceCountUtil.release(list); - } - @Benchmark public void testAddEntryV3() throws Exception { // Build the request and calculate the total size to be included in the packet. 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 d9545620163..04fa500d476 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 @@ -83,8 +83,9 @@ public void doSetup() throws Exception { data.writeBytes(randomBytes(entrySize)); digestBuf = ByteBufAllocator.DEFAULT.directBuffer(); - digestBuf.writeBytes(ByteBufList.coalesce( - dm.computeDigestAndPackageForSending(1234, 1234, entrySize, data))); + digestBuf.writeBytes((ByteBuf) + dm.computeDigestAndPackageForSending(1234, 1234, entrySize, data, + new byte[0], 0)); } } diff --git a/pom.xml b/pom.xml index e8227092bb3..f1926121fa1 100644 --- a/pom.xml +++ b/pom.xml @@ -176,6 +176,7 @@ 3.8.1 1.1.7.7 2.1.2 + 0.9.1 0.12 2.7 @@ -189,7 +190,7 @@ 0.8.8 1.8 3.1.0 - 3.1.2 + 3.2.1 3.2.0 3.10.1 3.0.2 @@ -801,6 +802,12 @@ rxjava ${rxjava.version} + + + com.carrotsearch + hppc + ${hppc.version} + @@ -1122,6 +1129,7 @@ ${redirectTestOutputToFile} ${forkCount.variable} false + false 1800 ${testRetryCount} diff --git a/site3/website/docs/admin/http.md b/site3/website/docs/admin/http.md index c5385d3fefb..73777ac1287 100644 --- a/site3/website/docs/admin/http.md +++ b/site3/website/docs/admin/http.md @@ -476,6 +476,47 @@ Currently all the HTTP endpoints could be divided into these 5 components: |503 | Bookie is not ready | * Body: <empty> +### Endpoint: /api/v1/bookie/entry_location_compact +1. Method: PUT + * Description: trigger entry location index rocksDB compact. Trigger all entry location rocksDB compact, if entryLocations not be specified. + * Parameters: + + | Name | Type | Required | Description | + |:-----|:-----|:---------|:------------| + |entryLocationRocksDBCompact | String | Yes | Configuration name(key) | + |entryLocations | String | no | entry location rocksDB path | + * Body: + ```json + { + "entryLocationRocksDBCompact": "true", + "entryLocations":"/data1/bookkeeper/ledgers/current/locations,/data2/bookkeeper/ledgers/current/locations" + } + ``` + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |405 | Method Not Allowed | + +2. Method: GET + * Description: All entry location index rocksDB compact status on bookie. true for is running. + * Response: + + | Code | Description | + |:-------|:------------| + |200 | Successful operation | + |403 | Permission denied | + |405 | Method Not Allowed | + * Body: + ```json + { + "/data1/bookkeeper/ledgers/current/locations" : true, + "/data2/bookkeeper/ledgers/current/locations" : false + } + ``` + ## Auto recovery diff --git a/site3/website/src/pages/release-notes.md b/site3/website/src/pages/release-notes.md index e6d45091851..dfa38d7691b 100644 --- a/site3/website/src/pages/release-notes.md +++ b/site3/website/src/pages/release-notes.md @@ -1,6 +1,59 @@ # Release notes +## 4.15.4 + +Release 4.15.4 includes multiple bug fixes and some dependencies CVE fixes. + +Apache BookKeeper users are encouraged to upgrade to 4.15.4. +The technical details of this release are summarized below. + +### Highlights + +#### Bugs + +* Fix QueueEntry recycle problem [PR #3747](https://github.com/apache/bookkeeper/pull/3747) +* Fix memory leak when the Bookie is in read only mode [PR #3746](https://github.com/apache/bookkeeper/pull/3746) +* Fix RegionAwareEnsemblePlacementPolicy.newEnsemble sometimes failed problem [PR #3725](https://github.com/apache/bookkeeper/pull/3725) +* Not wrap IOException twice form checkpoint [PR #3683](https://github.com/apache/bookkeeper/pull/3683) +* Make `jvm_memory_direct_bytes_used` metrics compatible with jdk8 [PR #3677](https://github.com/apache/bookkeeper/pull/3677) +* Fix memory leak when closeRecovered,failed on clearing newEnsemblesFromRecovery [PR #3672](https://github.com/apache/bookkeeper/pull/3672) +* Fix RegionAwareEnsemblePlacementPolicy update rack info problem [PR #3666](https://github.com/apache/bookkeeper/pull/3666) +* Exit bookkeeper shell correctly even if fails to run for some reason [PR #3663](https://github.com/apache/bookkeeper/pull/3663) +* Fix memory leak when operating ledger metadata [PR #3662](https://github.com/apache/bookkeeper/pull/3662) +* Check client if closed when complete callback [PR #3661](https://github.com/apache/bookkeeper/pull/3661) +* Fix issue where checkAllLedgers could get stuck when read throttling is enabled [PR #3655](https://github.com/apache/bookkeeper/pull/3655) +* Fix GetBookieInfo failed event stats [PR #3622](https://github.com/apache/bookkeeper/pull/3622) +* Apply recycle logic during add entry creation but ledger close to LedgerHandleAdv [PR #3621](https://github.com/apache/bookkeeper/pull/3621) +* BookieImpl remove wait until journal quits [PR #3603](https://github.com/apache/bookkeeper/pull/3603) + +#### Improvements + +* Support build in the aarch64 linux platform [PR #3828](https://github.com/apache/bookkeeper/pull/3828) +* Use ReferenceCountUtil.release() instead of ReferenceCountUtil.safeRelease() [PR #3797](https://github.com/apache/bookkeeper/pull/3797) +* Added api/v1/bookie/cluster_info REST API [PR #3713](https://github.com/apache/bookkeeper/pull/3713) +* Expose registrationClient in DefaultBookieAddressResolver [PR #3724](https://github.com/apache/bookkeeper/pull/3724) +* New ensemble choose different rack first [PR #3721](https://github.com/apache/bookkeeper/pull/3721) +* Improve bk_server.conf docs [PR #3715](https://github.com/apache/bookkeeper/pull/3715) +* Show result of isFenced in log [PR #3678](https://github.com/apache/bookkeeper/pull/3678) +* Include bkperf into bk all package [PR #3632](https://github.com/apache/bookkeeper/pull/3632) +* Add journal file path that caused failure in multi-journal config [PR #3623](https://github.com/apache/bookkeeper/pull/3623) +* Avoid extra buffer to prepend frame size [PR #3560](https://github.com/apache/bookkeeper/pull/3560) +* Using a separate thread pool to execute openWithMetadata [PR #3548](https://github.com/apache/bookkeeper/pull/3548) +* LedgerHandle: do not complete metadata operation on the ZookKeeper/Metadata callback thread [PR #3516](https://github.com/apache/bookkeeper/pull/3516) +* ledgerFragment check and results keep order [PR #3504](https://github.com/apache/bookkeeper/pull/3504) +* Simplified No network topology script is found default log stack output [PR #3496](https://github.com/apache/bookkeeper/pull/3496) +* Support apple m1 build [PR #3175](https://github.com/apache/bookkeeper/pull/3175) + +#### Dependency updates + +* Bump vertx-web from 4.3.2 to 4.3.8 [PR #3775](https://github.com/apache/bookkeeper/pull/3775) +* Upgrade docker image version to fix CVEs [PR #3640](https://github.com/apache/bookkeeper/pull/3640) +* Bump jcommander from 1.78 to 1.82 [PR #3476](https://github.com/apache/bookkeeper/pull/3476) + +### Details +https://github.com/apache/bookkeeper/pulls?page=1&q=is%3Apr+label%3Arelease%2F4.15.4+is%3Aclosed + ## 4.14.7 Release 4.14.7 includes multiple bug fixes. diff --git a/site3/website/versioned_docs/version-4.15.3/admin/autorecovery.md b/site3/website/versioned_docs/version-4.15.4/admin/autorecovery.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/admin/autorecovery.md rename to site3/website/versioned_docs/version-4.15.4/admin/autorecovery.md diff --git a/site3/website/versioned_docs/version-4.15.3/admin/bookies.md b/site3/website/versioned_docs/version-4.15.4/admin/bookies.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/admin/bookies.md rename to site3/website/versioned_docs/version-4.15.4/admin/bookies.md diff --git a/site3/website/versioned_docs/version-4.15.3/admin/decomission.md b/site3/website/versioned_docs/version-4.15.4/admin/decomission.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/admin/decomission.md rename to site3/website/versioned_docs/version-4.15.4/admin/decomission.md diff --git a/site3/website/versioned_docs/version-4.15.3/admin/geo-replication.md b/site3/website/versioned_docs/version-4.15.4/admin/geo-replication.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/admin/geo-replication.md rename to site3/website/versioned_docs/version-4.15.4/admin/geo-replication.md diff --git a/site3/website/versioned_docs/version-4.15.3/admin/http.md b/site3/website/versioned_docs/version-4.15.4/admin/http.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/admin/http.md rename to site3/website/versioned_docs/version-4.15.4/admin/http.md diff --git a/site3/website/versioned_docs/version-4.15.3/admin/metrics.md b/site3/website/versioned_docs/version-4.15.4/admin/metrics.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/admin/metrics.md rename to site3/website/versioned_docs/version-4.15.4/admin/metrics.md diff --git a/site3/website/versioned_docs/version-4.15.3/admin/perf.md b/site3/website/versioned_docs/version-4.15.4/admin/perf.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/admin/perf.md rename to site3/website/versioned_docs/version-4.15.4/admin/perf.md diff --git a/site3/website/versioned_docs/version-4.15.3/admin/placement.md b/site3/website/versioned_docs/version-4.15.4/admin/placement.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/admin/placement.md rename to site3/website/versioned_docs/version-4.15.4/admin/placement.md diff --git a/site3/website/versioned_docs/version-4.15.3/admin/upgrade.md b/site3/website/versioned_docs/version-4.15.4/admin/upgrade.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/admin/upgrade.md rename to site3/website/versioned_docs/version-4.15.4/admin/upgrade.md diff --git a/site3/website/versioned_docs/version-4.15.3/api/distributedlog-api.md b/site3/website/versioned_docs/version-4.15.4/api/distributedlog-api.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/api/distributedlog-api.md rename to site3/website/versioned_docs/version-4.15.4/api/distributedlog-api.md diff --git a/site3/website/versioned_docs/version-4.15.3/api/ledger-adv-api.md b/site3/website/versioned_docs/version-4.15.4/api/ledger-adv-api.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/api/ledger-adv-api.md rename to site3/website/versioned_docs/version-4.15.4/api/ledger-adv-api.md diff --git a/site3/website/versioned_docs/version-4.15.3/api/ledger-api.md b/site3/website/versioned_docs/version-4.15.4/api/ledger-api.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/api/ledger-api.md rename to site3/website/versioned_docs/version-4.15.4/api/ledger-api.md diff --git a/site3/website/versioned_docs/version-4.15.3/api/overview.md b/site3/website/versioned_docs/version-4.15.4/api/overview.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/api/overview.md rename to site3/website/versioned_docs/version-4.15.4/api/overview.md diff --git a/site3/website/versioned_docs/version-4.15.3/deployment/kubernetes.md b/site3/website/versioned_docs/version-4.15.4/deployment/kubernetes.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/deployment/kubernetes.md rename to site3/website/versioned_docs/version-4.15.4/deployment/kubernetes.md diff --git a/site3/website/versioned_docs/version-4.15.3/deployment/manual.md b/site3/website/versioned_docs/version-4.15.4/deployment/manual.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/deployment/manual.md rename to site3/website/versioned_docs/version-4.15.4/deployment/manual.md diff --git a/site3/website/versioned_docs/version-4.15.3/development/codebase.md b/site3/website/versioned_docs/version-4.15.4/development/codebase.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/development/codebase.md rename to site3/website/versioned_docs/version-4.15.4/development/codebase.md diff --git a/site3/website/versioned_docs/version-4.15.3/development/protocol.md b/site3/website/versioned_docs/version-4.15.4/development/protocol.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/development/protocol.md rename to site3/website/versioned_docs/version-4.15.4/development/protocol.md diff --git a/site3/website/versioned_docs/version-4.15.3/getting-started/concepts.md b/site3/website/versioned_docs/version-4.15.4/getting-started/concepts.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/getting-started/concepts.md rename to site3/website/versioned_docs/version-4.15.4/getting-started/concepts.md diff --git a/site3/website/versioned_docs/version-4.15.3/getting-started/installation.md b/site3/website/versioned_docs/version-4.15.4/getting-started/installation.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/getting-started/installation.md rename to site3/website/versioned_docs/version-4.15.4/getting-started/installation.md diff --git a/site3/website/versioned_docs/version-4.15.3/getting-started/run-locally.md b/site3/website/versioned_docs/version-4.15.4/getting-started/run-locally.md similarity index 100% rename from site3/website/versioned_docs/version-4.15.3/getting-started/run-locally.md rename to site3/website/versioned_docs/version-4.15.4/getting-started/run-locally.md diff --git a/site3/website/versioned_docs/version-4.15.3/overview/overview.md b/site3/website/versioned_docs/version-4.15.4/overview/overview.md similarity index 99% rename from site3/website/versioned_docs/version-4.15.3/overview/overview.md rename to site3/website/versioned_docs/version-4.15.4/overview/overview.md index b6dc7135a49..4d2351546b3 100644 --- a/site3/website/versioned_docs/version-4.15.3/overview/overview.md +++ b/site3/website/versioned_docs/version-4.15.4/overview/overview.md @@ -1,6 +1,6 @@ --- id: overview -title: Apache BookKeeper 4.15.3 +title: Apache BookKeeper 4.15.4 ---