You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@zookeeper.apache.org by GitBox <gi...@apache.org> on 2022/12/02 11:20:38 UTC

[GitHub] [zookeeper] anmolnar commented on a diff in pull request #1943: ZOOKEEPER-4570: Admin server API for taking snapshot and stream out data

anmolnar commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1038021488


##########
zookeeper-docs/src/main/resources/markdown/zookeeperAdmin.md:
##########
@@ -1208,7 +1208,32 @@ property, when available, is noted below.
 
     The default value is false.
 
-
+* *serializeLastProcessedZxid.enabled*
+  (Jave system property: **zookeeper.serializeLastProcessedZxid.enabled**)

Review Comment:
   If this option is backward and forward compatible, is there any reason why somebody wanted to turn it off?



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/util/RateLimiter.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.zookeeper.server.util;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * A class that provides simple interval-based rate limiting implementation.
+ */
+public class RateLimiter {
+    private final int rate;
+    private final long intervalInMs;
+    private long lastTimeReset;
+    private final AtomicInteger remained;
+
+    public RateLimiter(final int rate, final long interval, final TimeUnit unit) {
+        this.rate = rate;
+        this.intervalInMs = unit.toMillis(interval);
+        this.lastTimeReset = System.currentTimeMillis();

Review Comment:
   Please use ZooKeeper `Time.java` class for tracking elapsed time. It uses System.nanoTime() internally which is more reliably.



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/Commands.java:
##########
@@ -496,6 +506,87 @@ public CommandResponse run(ZooKeeperServer zkServer, Map<String, String> kwargs)
 
     }
 
+    /**
+     * Take a snapshot of current server and stream out the data.
+     *
+     *  Argument:
+     *   - "streaming": optional String to indicate whether streaming out data
+     *
+     *  Returned snapshot as stream if streaming is true and metadata of the snapshot
+     *   - "last_zxid": String
+     *   - "snapshot_size": String
+     */
+    public static class SnapshotCommand extends CommandBase {
+        static final String REQUEST_QUERY_PARAM_STREAMING = "streaming";
+
+        static final String RESPONSE_HEADER_LAST_ZXID = "last_zxid";
+        static final String RESPONSE_HEADER_SNAPSHOT_SIZE = "snapshot_size";
+
+        static final String ADMIN_SNAPSHOT_ENABLED = "zookeeper.admin.snapshot.enabled";
+        static final String ADMIN_SNAPSHOT_INTERVAL = "zookeeper.admin.snapshot.intervalInMS";
+
+        private static final long snapshotInterval = Integer.parseInt(System.getProperty(ADMIN_SNAPSHOT_INTERVAL, "300000"));
+
+        private final RateLimiter rateLimiter;
+
+        public SnapshotCommand() {
+            super(Arrays.asList("snapshot", "snap"));
+            rateLimiter = new RateLimiter(1, snapshotInterval, TimeUnit.MICROSECONDS);
+        }
+
+        @SuppressFBWarnings(value = "OBL_UNSATISFIED_OBLIGATION",
+                            justification = "FileInputStream is passed to CommandResponse and closed in StreamOutputter")
+        @Override
+        public CommandResponse run(final ZooKeeperServer zkServer, final Map<String, String> kwargs) {
+            final CommandResponse response = initializeResponse();
+
+            // check feature flag
+            final boolean snapshotEnabled = Boolean.parseBoolean(System.getProperty(ADMIN_SNAPSHOT_ENABLED, "false"));
+            if (!snapshotEnabled) {
+                response.setStatusCode(HttpServletResponse.SC_SERVICE_UNAVAILABLE);
+                LOG.warn("Snapshot command is disabled");
+                return response;
+            }
+
+            // check rate limiting
+            if (!rateLimiter.allow()) {
+                response.setStatusCode(429);

Review Comment:
   You can use here:
   ```java
   org.eclipse.jetty.http.HttpStatus.TOO_MANY_REQUESTS_429
   ```
   I see you prefer to use `HttpServletResponse` constants, but since the response code is a standard int, I think it doesn't matter.



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/StreamOutputter.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.zookeeper.server.admin;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import org.apache.zookeeper.common.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A class for streaming data out.
+ */
+public class StreamOutputter implements CommandOutputter{
+    private static final Logger LOG = LoggerFactory.getLogger(StreamOutputter.class);
+    private final String clientIP;
+
+    public StreamOutputter(final String clientIP) {
+        this.clientIP = clientIP;
+    }
+
+    @Override
+    public String getContentType() {
+        return "application/octet-stream";
+    }
+
+    @Override
+    public void output(final CommandResponse response, final OutputStream os) {
+        try (final InputStream is = response.getInputStream()){
+            IOUtils.copyBytes(is, os, 1024, true);
+        } catch (final IOException e) {
+            LOG.error("Exception occurred when streaming out data to {}", clientIP, e);

Review Comment:
   Logging the IP address is a useful addition to the error message. Have you considered adding to the `JsonOutputter` too?



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/IPAuthenticationProvider.java:
##########
@@ -128,4 +131,18 @@ public boolean isValid(String id) {
         return true;
     }
 
+    /**
+     * Returns the HTTP(s) client IP address
+     * @param request HttpServletRequest
+     * @return IP address
+     */
+    public static String getClientIPAddress(final HttpServletRequest request) {
+        // to handle the case that a HTTP(s) client connects via a proxy or load balancer
+        final String xForwardedForHeader = request.getHeader(X_FORWARDED_FOR_HEADER_NAME);
+        if (xForwardedForHeader == null) {
+            return request.getRemoteAddr();
+        }
+        // the format of the field is: X-Forwarded-For: client, proxy1, proxy2 ...
+        return new StringTokenizer(xForwardedForHeader, ",").nextToken().trim();
+    }

Review Comment:
   I don't think this method belongs to this class. `IPAuthenticationProvider` is responsible for authentication ZooKeeper client by the client IP address. You don't use any member of the class, neither instantiate it (static method), so please move it where the usage is (StreamOutputter.java).



-- 
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: notifications-unsubscribe@zookeeper.apache.org

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