You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by mm...@apache.org on 2019/05/03 16:29:29 UTC

[bookkeeper] branch master updated: Added HTTP handler to expose bookie state

This is an automated email from the ASF dual-hosted git repository.

mmerli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new 10859af  Added HTTP handler to expose bookie state
10859af is described below

commit 10859afb7e8f2ee095e70f4e5482959bb5a3067a
Author: Matteo Merli <mm...@apache.org>
AuthorDate: Fri May 3 09:29:24 2019 -0700

    Added HTTP handler to expose bookie state
    
    ### Motivation
    
    Allow to check the state of the bookie through HTTP.
    
    The main reason is to allow light-weight probes that can be run frequently to establish whether the bookie is running and meets a minimum readiness criteria. One example is to automatically wait until the bookie has completed the startup sequence.
    
    ### Changes
    
    Added new HTTP handler that exposes the values from the bookie `StateManager`.
    
    Reviewers: Ivan Kelly <iv...@apache.org>, Enrico Olivelli <eo...@gmail.com>, Jia Zhai <zh...@apache.org>, Sijie Guo <si...@apache.org>
    
    This closes #1995 from merlimat/bookie-state
---
 .../org/apache/bookkeeper/http/HttpRouter.java     |  4 +
 .../org/apache/bookkeeper/http/HttpServer.java     |  5 +-
 .../server/http/BKHttpServiceProvider.java         |  9 +++
 .../server/http/service/BookieIsReadyService.java  | 61 ++++++++++++++
 .../server/http/service/BookieStateService.java    | 92 ++++++++++++++++++++++
 .../bookkeeper/server/http/TestHttpService.java    | 42 +++++++++-
 site/docs/latest/admin/http.md                     | 34 ++++++++
 7 files changed, 245 insertions(+), 2 deletions(-)

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 b4251d7..99d5be6 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
@@ -47,6 +47,8 @@ public abstract class HttpRouter<Handler> {
     public static final String EXPAND_STORAGE               = "/api/v1/bookie/expand_storage";
     public static final String GC                           = "/api/v1/bookie/gc";
     public static final String GC_DETAILS                   = "/api/v1/bookie/gc_details";
+    public static final String BOOKIE_STATE                 = "/api/v1/bookie/state";
+    public static final String BOOKIE_IS_READY              = "/api/v1/bookie/is_ready";
     // autorecovery
     public static final String RECOVERY_BOOKIE              = "/api/v1/autorecovery/bookie";
     public static final String LIST_UNDER_REPLICATED_LEDGER = "/api/v1/autorecovery/list_under_replicated_ledger";
@@ -77,6 +79,8 @@ public abstract class HttpRouter<Handler> {
         this.endpointHandlers.put(EXPAND_STORAGE, handlerFactory.newHandler(HttpServer.ApiType.EXPAND_STORAGE));
         this.endpointHandlers.put(GC, handlerFactory.newHandler(HttpServer.ApiType.GC));
         this.endpointHandlers.put(GC_DETAILS, handlerFactory.newHandler(HttpServer.ApiType.GC_DETAILS));
+        this.endpointHandlers.put(BOOKIE_STATE, handlerFactory.newHandler(HttpServer.ApiType.BOOKIE_STATE));
+        this.endpointHandlers.put(BOOKIE_IS_READY, handlerFactory.newHandler(HttpServer.ApiType.BOOKIE_IS_READY));
 
         // autorecovery
         this.endpointHandlers.put(RECOVERY_BOOKIE, handlerFactory.newHandler(HttpServer.ApiType.RECOVERY_BOOKIE));
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 0922b56..c694a07 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
@@ -35,7 +35,8 @@ public interface HttpServer {
         REDIRECT(302),
         FORBIDDEN(403),
         NOT_FOUND(404),
-        INTERNAL_ERROR(500);
+        INTERNAL_ERROR(500),
+        SERVICE_UNAVAILABLE(503);
 
         private int value;
 
@@ -79,6 +80,8 @@ public interface HttpServer {
         EXPAND_STORAGE,
         GC,
         GC_DETAILS,
+        BOOKIE_STATE,
+        BOOKIE_IS_READY,
 
         // autorecovery
         RECOVERY_BOOKIE,
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 662777c..24795e5 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
@@ -19,10 +19,13 @@
 package org.apache.bookkeeper.server.http;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
 import java.io.IOException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
+
 import lombok.extern.slf4j.Slf4j;
+
 import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeperAdmin;
@@ -37,6 +40,8 @@ import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
 import org.apache.bookkeeper.proto.BookieServer;
 import org.apache.bookkeeper.replication.Auditor;
 import org.apache.bookkeeper.replication.AutoRecoveryMain;
+import org.apache.bookkeeper.server.http.service.BookieIsReadyService;
+import org.apache.bookkeeper.server.http.service.BookieStateService;
 import org.apache.bookkeeper.server.http.service.ConfigurationService;
 import org.apache.bookkeeper.server.http.service.DecommissionService;
 import org.apache.bookkeeper.server.http.service.DeleteLedgerService;
@@ -214,6 +219,10 @@ public class BKHttpServiceProvider implements HttpServiceProvider {
                 return new TriggerGCService(configuration, bookieServer);
             case GC_DETAILS:
                 return new GCDetailsService(configuration, bookieServer);
+            case BOOKIE_STATE:
+                return new BookieStateService(bookieServer.getBookie());
+            case BOOKIE_IS_READY:
+                return new BookieIsReadyService(bookieServer.getBookie());
 
             // autorecovery
             case RECOVERY_BOOKIE:
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/BookieIsReadyService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/BookieIsReadyService.java
new file mode 100644
index 0000000..04e13b6
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/BookieIsReadyService.java
@@ -0,0 +1,61 @@
+/*
+ * 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 org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.StateManager;
+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;
+
+/**
+ * HttpEndpointService that returns 200 if the bookie is ready.
+ */
+public class BookieIsReadyService implements HttpEndpointService {
+
+    private final Bookie bookie;
+
+    public BookieIsReadyService(Bookie bookie) {
+        this.bookie = checkNotNull(bookie);
+    }
+
+    @Override
+    public HttpServiceResponse handle(HttpServiceRequest request) throws Exception {
+        HttpServiceResponse response = new HttpServiceResponse();
+
+        if (HttpServer.Method.GET != request.getMethod()) {
+            response.setCode(HttpServer.StatusCode.NOT_FOUND);
+            response.setBody("Only support GET method check if bookie is ready.");
+            return response;
+        }
+
+        StateManager sm = bookie.getStateManager();
+        if (sm.isRunning() && !sm.isShuttingDown()) {
+            response.setCode(HttpServer.StatusCode.OK);
+            response.setBody("");
+        } else {
+            response.setCode(HttpServer.StatusCode.SERVICE_UNAVAILABLE);
+            response.setBody("Bookie is not fully started yet");
+        }
+        return response;
+    }
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/BookieStateService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/BookieStateService.java
new file mode 100644
index 0000000..45ac2c3
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/http/service/BookieStateService.java
@@ -0,0 +1,92 @@
+/*
+ * 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 lombok.Data;
+import lombok.NoArgsConstructor;
+
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.StateManager;
+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;
+
+/**
+ * HttpEndpointService that exposes the current state of the bookie.
+ *
+ * <p>Get the current bookie status:
+ *
+ * <pre>
+ * <code>
+ * {
+ *  "running" : true,
+ *  "readOnly" : false,
+ *  "shuttingDown" : false,
+ *  "availableForHighPriorityWrites" : true
+ *}
+ * </code>
+ * </pre>
+ */
+public class BookieStateService implements HttpEndpointService {
+
+    private final Bookie bookie;
+
+    public BookieStateService(Bookie bookie) {
+        this.bookie = checkNotNull(bookie);
+    }
+
+    /**
+     * POJO definition for the bookie state response.
+     */
+    @Data
+    @NoArgsConstructor
+    public static class BookieState {
+        private boolean running;
+        private boolean readOnly;
+        private boolean shuttingDown;
+        private boolean availableForHighPriorityWrites;
+    }
+
+    @Override
+    public HttpServiceResponse handle(HttpServiceRequest request) throws Exception {
+        HttpServiceResponse response = new HttpServiceResponse();
+
+        if (HttpServer.Method.GET != request.getMethod()) {
+            response.setCode(HttpServer.StatusCode.NOT_FOUND);
+            response.setBody("Only support GET method to retrieve bookie state.");
+            return response;
+        }
+
+        StateManager sm = bookie.getStateManager();
+        BookieState bs = new BookieState();
+        bs.running = sm.isRunning();
+        bs.readOnly = sm.isReadOnly();
+        bs.shuttingDown = sm.isShuttingDown();
+        bs.availableForHighPriorityWrites = sm.isAvailableForHighPriorityWrites();
+
+        String jsonResponse = JsonUtil.toJson(bs);
+        response.setBody(jsonResponse);
+        response.setCode(HttpServer.StatusCode.OK);
+        return response;
+    }
+}
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 ee6b278..02aa42b 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
@@ -26,12 +26,15 @@ import static org.junit.Assert.assertTrue;
 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.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.concurrent.Future;
+
 import lombok.Cleanup;
+
 import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.ClientUtil;
 import org.apache.bookkeeper.client.LedgerHandle;
@@ -48,8 +51,8 @@ import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.LedgerUnderreplicationManager;
 import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.replication.AuditorElector;
+import org.apache.bookkeeper.server.http.service.BookieStateService.BookieState;
 import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
-
 import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -826,4 +829,41 @@ public class TestHttpService extends BookKeeperClusterTestCase {
         HttpServiceResponse response3 = gcDetailsService.handle(request3);
         assertEquals(HttpServer.StatusCode.NOT_FOUND.getValue(), response3.getStatusCode());
     }
+
+    @Test
+    public void testGetBookieState() throws Exception {
+        HttpEndpointService bookieStateServer = bkHttpServiceProvider
+                .provideHttpEndpointService(HttpServer.ApiType.BOOKIE_STATE);
+
+        HttpServiceRequest request1 = new HttpServiceRequest(null, HttpServer.Method.GET, null);
+        HttpServiceResponse response1 = bookieStateServer.handle(request1);
+        assertEquals(HttpServer.StatusCode.OK.getValue(), response1.getStatusCode());
+
+        BookieState bs = JsonUtil.fromJson(response1.getBody(), BookieState.class);
+        assertEquals(true, bs.isRunning());
+        assertEquals(false, bs.isReadOnly());
+        assertEquals(true, bs.isAvailableForHighPriorityWrites());
+        assertEquals(false, bs.isShuttingDown());
+    }
+
+    @Test
+    public void testGetBookieIsReady() throws Exception {
+        HttpEndpointService bookieStateServer = bkHttpServiceProvider
+                .provideHttpEndpointService(HttpServer.ApiType.BOOKIE_IS_READY);
+
+        HttpServiceRequest request1 = new HttpServiceRequest(null, HttpServer.Method.GET, null);
+        HttpServiceResponse response1 = bookieStateServer.handle(request1);
+        assertEquals(HttpServer.StatusCode.OK.getValue(), response1.getStatusCode());
+
+        // Try using POST instead of GET
+        HttpServiceRequest request2 = new HttpServiceRequest(null, HttpServer.Method.POST, null);
+        HttpServiceResponse response2 = bookieStateServer.handle(request2);
+        assertEquals(HttpServer.StatusCode.NOT_FOUND.getValue(), response2.getStatusCode());
+
+        // Simulate bookies shutting down
+        bs.forEach(bookieServer -> bookieServer.getBookie().getStateManager().forceToShuttingDown());
+        HttpServiceRequest request3 = new HttpServiceRequest(null, HttpServer.Method.GET, null);
+        HttpServiceResponse response3 = bookieStateServer.handle(request3);
+        assertEquals(HttpServer.StatusCode.SERVICE_UNAVAILABLE.getValue(), response3.getStatusCode());
+    }
 }
diff --git a/site/docs/latest/admin/http.md b/site/docs/latest/admin/http.md
index 270dcb3..c7fcf0c 100644
--- a/site/docs/latest/admin/http.md
+++ b/site/docs/latest/admin/http.md
@@ -321,6 +321,40 @@ Currently all the HTTP endpoints could be divided into these 4 components:
         } ]
        ```
 
+### Endpoint: /api/v1/bookie/state
+1. Method: GET
+   * Description:  Exposes the current state of bookie
+   * Response:
+
+        | Code   | Description |
+        |:-------|:------------|
+        |200 | Successful operation |
+        |403 | Permission denied |
+        |404 | Not found |
+   * Body:
+      ```json
+      {
+         "running" : true,
+         "readOnly" : false,
+         "shuttingDown" : false,
+         "availableForHighPriorityWrites" : true
+       }
+      ```
+
+### Endpoint: /api/v1/bookie/is_ready
+1. Method: GET
+   * Description:  Return true if the bookie is ready
+   * Response:
+
+        | Code   | Description |
+        |:-------|:------------|
+        |200 | Successful operation |
+        |403 | Permission denied |
+        |404 | Not found |
+        |503 | Bookie is not ready |
+   * Body: <empty>
+
+
 ## Auto recovery
 
 ### Endpoint: /api/v1/autorecovery/bookie/