You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by "horizonzy (via GitHub)" <gi...@apache.org> on 2023/03/14 09:34:03 UTC

[GitHub] [bookkeeper] horizonzy commented on a diff in pull request #3802: Add trigger entry location index rocksDB compact REST API

horizonzy commented on code in PR #3802:
URL: https://github.com/apache/bookkeeper/pull/3802#discussion_r1135246248


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java:
##########
@@ -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);

Review Comment:
   We can add the path in the log content.



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/TriggerLocationCompactService.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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 java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+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.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * HttpEndpointService that handle force trigger entry location compact requests.
+ *
+ * <p>The PUT method will trigger entry location compact on current bookie.
+ *
+ * <p>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;
+
+    public TriggerLocationCompactService(BookieServer bookieServer) {
+        this.bookieServer = checkNotNull(bookieServer);
+    }
+
+    @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<String, Object> configMap = JsonUtil.fromJson(requestBody, HashMap.class);
+                Boolean isEntryLocationCompact = (Boolean) configMap
+                        .getOrDefault("entryLocationRocksDBCompact", false);
+                String entryLocations = (String) configMap.getOrDefault("entryLocations", "");
+                List<String> entryLocationDBPath = ledgerStorage.getEntryLocationDBPath();
+                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.
+                    List<String> locations = Arrays.asList(entryLocations.trim().split(","));
+                    Map<String, Boolean> compactStatus = ledgerStorage.isEntryLocationCompacting(locations);
+
+                    if (entryLocationDBPath.containsAll(locations) && !compactStatus.containsValue(true)) {
+                        ledgerStorage.entryLocationCompact(locations);
+                        output = String.format("Triggered entry Location RocksDB: %s compact on bookie:%s.",
+                                entryLocations, bookieServer.getBookieId());
+                        response.setCode(HttpServer.StatusCode.OK);
+                    } else if (entryLocationDBPath.containsAll(locations)) {
+                        // If any specified entry location RocksDB is running, not trigger compact again.
+                        output = String.format("Specified entry location RocksDB: %s already running compact,"
+                                + " compact status: %s.", entryLocations, compactStatus);
+                        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.
+                    boolean isCompactRunning = ledgerStorage.isEntryLocationCompacting();

Review Comment:
   We should trigger the compact which is not compacting. `path1`, `path2`, `path3`.
   `path1`, and `path2` is compacting. we should ignore `path1` and `path2`. Trigger `path3` compact
   



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/TriggerLocationCompactService.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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 java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+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.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * HttpEndpointService that handle force trigger entry location compact requests.
+ *
+ * <p>The PUT method will trigger entry location compact on current bookie.
+ *
+ * <p>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;
+
+    public TriggerLocationCompactService(BookieServer bookieServer) {
+        this.bookieServer = checkNotNull(bookieServer);
+    }
+
+    @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<String, Object> configMap = JsonUtil.fromJson(requestBody, HashMap.class);
+                Boolean isEntryLocationCompact = (Boolean) configMap
+                        .getOrDefault("entryLocationRocksDBCompact", false);
+                String entryLocations = (String) configMap.getOrDefault("entryLocations", "");
+                List<String> entryLocationDBPath = ledgerStorage.getEntryLocationDBPath();
+                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.
+                    List<String> locations = Arrays.asList(entryLocations.trim().split(","));
+                    Map<String, Boolean> compactStatus = ledgerStorage.isEntryLocationCompacting(locations);
+
+                    if (entryLocationDBPath.containsAll(locations) && !compactStatus.containsValue(true)) {

Review Comment:
   The judgment is a little strange. If there are `path1`, `path2`, and `path3` in bookie.
   
   The request input locations are `path1`, and `path4`. It won't trigger.
   The request input locations are `path1`, and `path2`. And the `path1` is compacting, it also won't trigger.
   



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/TriggerLocationCompactService.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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 java.util.HashMap;
+import java.util.Map;
+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.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * HttpEndpointService that handle force trigger entry location compact requests.
+ *
+ * <p>The PUT method will trigger entry location compact on current bookie.
+ *
+ * <p>The GET method will get the entry location compact running or not.
+ * Output would be like:
+ *        {
+ *           "is_entry_location_compact" : "false"
+ *        }
+ */
+
+public class TriggerLocationCompactService implements HttpEndpointService {
+
+    static final Logger LOG = LoggerFactory.getLogger(TriggerLocationCompactService.class);
+
+    private final BookieServer bookieServer;
+
+    public TriggerLocationCompactService(BookieServer bookieServer) {
+        this.bookieServer = checkNotNull(bookieServer);
+    }
+
+    @Override
+    public HttpServiceResponse handle(HttpServiceRequest request) throws Exception {
+        HttpServiceResponse response = new HttpServiceResponse();
+
+        if (HttpServer.Method.PUT.equals(request.getMethod())) {
+            String requestBody = request.getBody();
+            String output = "Not trigger Entry Location RocksDB compact.";
+            if (StringUtils.isNotBlank(requestBody)) {
+                @SuppressWarnings("unchecked")
+                Map<String, Object> configMap = JsonUtil.fromJson(requestBody, HashMap.class);

Review Comment:
   We would better catch Exception.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org