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/11/08 22:50:30 UTC

[GitHub] [zookeeper] li4wang opened a new pull request, #1943: ZOOKEEPER-4570: Admin server API for taking snapshot and stream out data

li4wang opened a new pull request, #1943:
URL: https://github.com/apache/zookeeper/pull/1943

   Provides a snapshot command for taking snapshot and streaming out data
   
   Author: Li Wang <li...@apple.com>


-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#issuecomment-1371249655

   Great, thanks @eolivelli


-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#issuecomment-1338977186

   @anmolnar thanks for the additional comments. They have been addressed. Would you mind taking a look at them? Thanks


-- 
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


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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1038076604


##########
zookeeper-server/src/test/java/org/apache/zookeeper/server/util/RateLimiterTest.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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 static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import java.util.concurrent.TimeUnit;
+import org.junit.jupiter.api.Test;
+
+public class RateLimiterTest {
+
+    @Test
+    public void testAllow_withinInterval() {
+        final int rate = 2;
+        final RateLimiter rateLimiter = new RateLimiter(rate, 5, TimeUnit.SECONDS);
+        for (int i = 0; i < rate; i++) {
+            assertTrue(rateLimiter.allow());
+        }
+        assertFalse(rateLimiter.allow());
+    }
+
+    @Test
+    public void testAllow_exceedInterval() throws Exception {
+        final int interval = 1;
+
+        final RateLimiter rateLimiter = new RateLimiter(1, interval, TimeUnit.SECONDS);
+        assertTrue(rateLimiter.allow());
+        assertFalse(rateLimiter.allow());
+        Thread.sleep(TimeUnit.SECONDS.toMillis(interval) + 1);

Review Comment:
   Please wait 2 seconds at least. Thread.sleep() in tests could easly lead to flaky test.
   Additonally I think it would be nice to have a multithreaded test as well: start 10 worker in an executor to grab the numbers and verify that the 11th is disallowed within the time interval.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#issuecomment-1307945320

   This work is built on top of the work done in  https://issues.apache.org/jira/browse/ZOOKEEPER-3499. Thanks @maoling for his great contribution.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#issuecomment-1335785734

   @anmolnar thanks a lot for the comments. They have been addressed. Would you mind taking a quick look at them?


-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1040568525


##########
zookeeper-server/src/test/java/org/apache/zookeeper/server/util/RateLimiterTest.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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 static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import org.junit.jupiter.api.Test;
+
+public class RateLimiterTest {
+
+    @Test
+    public void testAllow_withinInterval() {
+        final int rate = 2;
+        final RateLimiter rateLimiter = new RateLimiter(rate, 5, TimeUnit.SECONDS);
+        for (int i = 0; i < rate; i++) {
+            assertTrue(rateLimiter.allow());
+        }
+        assertFalse(rateLimiter.allow());
+    }
+
+    @Test
+    public void testAllow_withinInterval_multiThreaded() {
+        final int rate = 10;
+
+        final RateLimiter rateLimiter = new RateLimiter(rate, 5, TimeUnit.SECONDS);
+        final ThreadPoolExecutor executor = (ThreadPoolExecutor) Executors.newFixedThreadPool(rate + 1);
+        for (int i = 0; i < rate; i++) {
+            executor.execute(() -> assertTrue(rateLimiter.allow()));
+        }
+        executor.execute(() -> assertFalse(rateLimiter.allow()));
+    }
+
+    @Test
+    public void testAllow_exceedInterval() throws Exception {
+        final int interval = 1;
+
+        final RateLimiter rateLimiter = new RateLimiter(1, interval, TimeUnit.SECONDS);
+        assertTrue(rateLimiter.allow());
+        assertFalse(rateLimiter.allow());
+        Thread.sleep(TimeUnit.SECONDS.toMillis(interval) + 3);

Review Comment:
   Good catch!  I didn't realize the original issue was 1 ms more instead of 1 second more.  Fixed.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1040662947


##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java:
##########
@@ -1744,6 +1744,42 @@ public boolean deserializeZxidDigest(InputArchive ia, long startZxidOfSnapshot)
         }
     }
 
+    /**
+     * Serializes the lastProcessedZxid so we can get it from snapshot instead the snapshot file name.
+     * This is needed for performing snapshot and restore via admin server commands.
+     *
+     * @param oa the output stream to write to
+     * @return true if the lastProcessedZxid is serialized successfully, otherwise false
+     * @throws IOException
+     */
+    public boolean serializeLastProcessedZxid(final OutputArchive oa) throws IOException {
+        if (!ZooKeeperServer.isSerializeLastProcessedZxidEnabled()) {
+            return false;
+        }
+        oa.writeLong(lastProcessedZxid, "lastZxid");
+        return true;
+    }
+
+    /**
+     * Deserializes the lastProcessedZxid from the input stream and updates the lastProcessedZxid field.
+     *
+     * @param ia the input stream to read from
+     * @return true if lastProcessedZxid is deserialized successfully, otherwise false
+     * @throws IOException

Review Comment:
   *[EmptyBlockTag](https://google.github.io/styleguide/javaguide.html#s7.1.3-javadoc-block-tags):*  A block tag (@param, @return, @throws, @deprecated) has an empty description. Block tags without descriptions don't add much value for future readers of the code; consider removing the tag entirely or adding a description.
   
   ---
   
   
   ```suggestion
        *
   ```
   
   
   
   ---
   
   <details><summary><b>ℹ️ Learn about @sonatype-lift commands</b></summary>
   
   You can reply with the following commands. For example, reply with ***@sonatype-lift ignoreall*** to leave out all findings.
   | **Command** | **Usage** |
   | ------------- | ------------- |
   | `@sonatype-lift ignore` | Leave out the above finding from this PR |
   | `@sonatype-lift ignoreall` | Leave out all the existing findings from this PR |
   | `@sonatype-lift exclude <file\|issue\|path\|tool>` | Exclude specified `file\|issue\|path\|tool` from Lift findings by updating your config.toml file |
   
   **Note:** When talking to LiftBot, you need to **refresh** the page to see its response.
   <sub>[Click here](https://github.com/apps/sonatype-lift/installations/new) to add LiftBot to another repo.</sub></details>
   
   
   
   ---
   
   Was this a good recommendation?
   [ [πŸ™ Not relevant](https://www.sonatype.com/lift-comment-rating?comment=358311377&lift_comment_rating=1) ] - [ [πŸ˜• Won't fix](https://www.sonatype.com/lift-comment-rating?comment=358311377&lift_comment_rating=2) ] - [ [πŸ˜‘ Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=358311377&lift_comment_rating=3) ] - [ [πŸ™‚ Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=358311377&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=358311377&lift_comment_rating=5) ]



##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java:
##########
@@ -1744,6 +1744,42 @@ public boolean deserializeZxidDigest(InputArchive ia, long startZxidOfSnapshot)
         }
     }
 
+    /**
+     * Serializes the lastProcessedZxid so we can get it from snapshot instead the snapshot file name.
+     * This is needed for performing snapshot and restore via admin server commands.
+     *
+     * @param oa the output stream to write to
+     * @return true if the lastProcessedZxid is serialized successfully, otherwise false
+     * @throws IOException

Review Comment:
   *[EmptyBlockTag](https://google.github.io/styleguide/javaguide.html#s7.1.3-javadoc-block-tags):*  A block tag (@param, @return, @throws, @deprecated) has an empty description. Block tags without descriptions don't add much value for future readers of the code; consider removing the tag entirely or adding a description.
   
   ---
   
   
   ```suggestion
        *
   ```
   
   
   
   ---
   
   <details><summary><b>ℹ️ Learn about @sonatype-lift commands</b></summary>
   
   You can reply with the following commands. For example, reply with ***@sonatype-lift ignoreall*** to leave out all findings.
   | **Command** | **Usage** |
   | ------------- | ------------- |
   | `@sonatype-lift ignore` | Leave out the above finding from this PR |
   | `@sonatype-lift ignoreall` | Leave out all the existing findings from this PR |
   | `@sonatype-lift exclude <file\|issue\|path\|tool>` | Exclude specified `file\|issue\|path\|tool` from Lift findings by updating your config.toml file |
   
   **Note:** When talking to LiftBot, you need to **refresh** the page to see its response.
   <sub>[Click here](https://github.com/apps/sonatype-lift/installations/new) to add LiftBot to another repo.</sub></details>
   
   
   
   ---
   
   Was this a good recommendation?
   [ [πŸ™ Not relevant](https://www.sonatype.com/lift-comment-rating?comment=358311402&lift_comment_rating=1) ] - [ [πŸ˜• Won't fix](https://www.sonatype.com/lift-comment-rating?comment=358311402&lift_comment_rating=2) ] - [ [πŸ˜‘ Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=358311402&lift_comment_rating=3) ] - [ [πŸ™‚ Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=358311402&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=358311402&lift_comment_rating=5) ]



-- 
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


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

Posted by GitBox <gi...@apache.org>.
eolivelli commented on PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#issuecomment-1370681894

   @li4wang I have merged master branch into you branch in order to fix CI


-- 
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


[GitHub] [zookeeper] eolivelli merged pull request #1943: ZOOKEEPER-4570: Admin server API for taking snapshot and stream out data

Posted by GitBox <gi...@apache.org>.
eolivelli merged PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943


-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1038390377


##########
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:
   No, I don't see anyone wants to turn it off unless we run into unexpected issue, but it doesn't hurt to have optional available for now.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1020630119


##########
zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/SnapshotCommandTest.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.zookeeper.server.admin;
+
+import static org.apache.zookeeper.server.admin.Commands.SnapshotCommand.ADMIN_SNAPSHOT_ENABLED;
+import static org.apache.zookeeper.server.admin.Commands.SnapshotCommand.ADMIN_SNAPSHOT_INTERVAL;
+import static org.apache.zookeeper.server.admin.Commands.SnapshotCommand.REQUEST_QUERY_PARAM_STREAMING;
+import static org.apache.zookeeper.server.admin.JettyAdminServerTest.URL_FORMAT;
+import static org.apache.zookeeper.server.admin.JettyAdminServerTest.jettyAdminPort;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.UnsupportedEncodingException;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.nio.file.Files;
+import java.util.HashMap;
+import java.util.Map;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.PortAssignment;
+import org.apache.zookeeper.ZKTestCase;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.common.IOUtils;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.apache.zookeeper.server.ZooKeeperServer;
+import org.apache.zookeeper.test.ClientBase;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.api.io.TempDir;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@TestInstance(TestInstance.Lifecycle.PER_CLASS)
+public class SnapshotCommandTest extends ZKTestCase {
+    private static final Logger LOG = LoggerFactory.getLogger(SnapshotCommandTest.class);
+
+    private static final String PATH = "/snapshot_test";
+    private static final int NODE_COUNT = 10;
+
+    private final String hostPort =  "127.0.0.1:" + PortAssignment.unique();
+    private ServerCnxnFactory cnxnFactory;
+    private JettyAdminServer adminServer;
+    private ZooKeeperServer zks;
+    private ZooKeeper zk;
+
+    @TempDir
+    static File dataDir;
+
+    @TempDir
+    static File logDir;
+
+    @BeforeAll
+    public void setup() throws Exception {
+        // start ZookeeperServer
+        System.setProperty("zookeeper.4lw.commands.whitelist", "*");

Review Comment:
   Yes, all the system properties are cleared.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1039443157


##########
zookeeper-server/src/test/java/org/apache/zookeeper/server/util/RateLimiterTest.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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 static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import org.junit.jupiter.api.Test;
+
+public class RateLimiterTest {
+
+    @Test
+    public void testAllow_withinInterval() {
+        final int rate = 2;
+        final RateLimiter rateLimiter = new RateLimiter(rate, 5, TimeUnit.SECONDS);
+        for (int i = 0; i < rate; i++) {
+            assertTrue(rateLimiter.allow());
+        }
+        assertFalse(rateLimiter.allow());
+    }
+
+    @Test
+    public void testAllow_withinInterval_multiThreaded() {
+        final int rate = 10;
+
+        final RateLimiter rateLimiter = new RateLimiter(rate, 5, TimeUnit.SECONDS);
+        final ThreadPoolExecutor executor = (ThreadPoolExecutor) Executors.newFixedThreadPool(rate + 1);
+        for (int i = 0; i < rate; i++) {
+            executor.execute(() -> assertTrue(rateLimiter.allow()));
+        }
+        executor.execute(() -> assertFalse(rateLimiter.allow()));
+    }
+
+    @Test
+    public void testAllow_exceedInterval() throws Exception {
+        final int interval = 1;
+
+        final RateLimiter rateLimiter = new RateLimiter(1, interval, TimeUnit.SECONDS);
+        assertTrue(rateLimiter.allow());
+        assertFalse(rateLimiter.allow());
+        Thread.sleep(TimeUnit.SECONDS.toMillis(interval) + 3);

Review Comment:
   Wait 2 seconds (internal + 1), not 2 milliseconds more.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1038390377


##########
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:
   No, I don't see anyone wants to turn it off unless we run into unexpected issue, but I feel it doesn't hurt to have the option available for now.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1040607436


##########
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:
   Yes, you are right. Added the check and also the unit test.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1038424755


##########
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:
   Good idea. Changed.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1039441532


##########
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:
   Fine. Next time please add a refactoring to the upcoming patch instead. Looking at this patch alone means that you break the encapsulation principle.



##########
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:
   Yeah, it probably doesn't hurt, but we should not implement something just because it doesn't hurt. Also don't need to prepare feature flags for unexcpeted issues. These are not valid reasons. 
   
   Either way, you can leave it as it is. One more thing: the doc mentions that this option is required for the Admin's snapshot command to work properly. If that's the case the Admin command should check whether this option is enabled, but I don't see the check there, right?



-- 
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


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

Posted by GitBox <gi...@apache.org>.
eolivelli commented on PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#issuecomment-1340007866

   It looks like we have a problem with the C client unrelated to this patch.
   Maybe on the CI machines GH upgraded some tools.
   
   I will take a look tomorrow.
   
   If you want you can send the second patch on top of this branch  in the meantime 


-- 
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


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

Posted by GitBox <gi...@apache.org>.
eolivelli commented on PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#issuecomment-1339127494

   @li4wang I have restarted CI
   We will commit to master branch as soon as CI passes
   
   thanks!
   
   looking forward for the next patch


-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#issuecomment-1309540677

   > 
   [2022-11-08T23:44:39.202Z] [ERROR] testRevalidateCount  Time elapsed: 1.256 s  <<< ERROR!
   [2022-11-08T23:44:39.202Z] java.lang.NullPointerException
   [2022-11-08T23:44:39.202Z] 	at org.apache.zookeeper.test.QuorumUtil.getConnectionStringForServer(QuorumUtil.java:343)
   [2022-11-08T23:44:39.202Z] 	at org.apache.zookeeper.server.ConnectionMetricsTest.testRevalidateCount(ConnectionMetricsTest.java:65)
   
   The test failure is not related to this PR. It will be fixed by PR https://github.com/apache/zookeeper/pull/1944


-- 
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


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

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1017204225


##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java:
##########
@@ -535,23 +541,46 @@ public void loadData() throws IOException, InterruptedException {
         takeSnapshot();

Review Comment:
   πŸ’¬ 2 similar findings have been found in this PR
   
   ---
   
   *THREAD_SAFETY_VIOLATION:*  Unprotected write. Non-private method `ZooKeeperServer.loadData()` indirectly writes to field `this.zkDb.dataTree.RATE_LOGGER.count` outside of synchronization.
    Reporting because another access to the same memory occurs on a background thread, although this access may not.
   
   ---
   
   <details><summary><b>πŸ”Ž Expand here to view all instances of this finding</b></summary><br/>
   
   <div align="center">
   
   | **File Path** | **Line Number** |
   | ------------- | ------------- |
   | zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java | [567](https://github.com/li4wang/zookeeper/blob/a71842e8ded132c092c94a6e064d4714503cb108/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java#L567)|
   | zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java | [549](https://github.com/li4wang/zookeeper/blob/a71842e8ded132c092c94a6e064d4714503cb108/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java#L549)|
   <p><a href="https://lift.sonatype.com/results/github.com/apache/zookeeper/01GHCQ07666B32ET7QXDPQDD9Y?t=Infer|THREAD_SAFETY_VIOLATION" target="_blank">Visit the Lift Web Console</a> to find more details in your report.</p></div></details>
   
   
   
   ---
   
   <details><summary><b>ℹ️ Learn about @sonatype-lift commands</b></summary>
   
   You can reply with the following commands. For example, reply with ***@sonatype-lift ignoreall*** to leave out all findings.
   | **Command** | **Usage** |
   | ------------- | ------------- |
   | `@sonatype-lift ignore` | Leave out the above finding from this PR |
   | `@sonatype-lift ignoreall` | Leave out all the existing findings from this PR |
   | `@sonatype-lift exclude <file\|issue\|path\|tool>` | Exclude specified `file\|issue\|path\|tool` from Lift findings by updating your config.toml file |
   
   **Note:** When talking to LiftBot, you need to **refresh** the page to see its response.
   <sub>[Click here](https://github.com/apps/sonatype-lift/installations/new) to add LiftBot to another repo.</sub></details>
   
   
   
   ---
   
   Was this a good recommendation?
   [ [πŸ™ Not relevant](https://www.sonatype.com/lift-comment-rating?comment=351724297&lift_comment_rating=1) ] - [ [πŸ˜• Won't fix](https://www.sonatype.com/lift-comment-rating?comment=351724297&lift_comment_rating=2) ] - [ [πŸ˜‘ Not critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=351724297&lift_comment_rating=3) ] - [ [πŸ™‚ Critical, will fix](https://www.sonatype.com/lift-comment-rating?comment=351724297&lift_comment_rating=4) ] - [ [😊 Critical, fixing now](https://www.sonatype.com/lift-comment-rating?comment=351724297&lift_comment_rating=5) ]



-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1038419979


##########
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:
   Yes, changed.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
anmolnar commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1038044021


##########
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 reliable.



##########
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 authenticating 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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#issuecomment-1312342748

   @eolivelli thanks for the comments. They have been addressed. Would you mind taking a quick look up them? Thanks


-- 
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


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

Posted by GitBox <gi...@apache.org>.
eolivelli commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1020168886


##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/StreamOutputter.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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);
+
+    public StreamOutputter() {
+    }
+
+    @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", e);

Review Comment:
   follow up enhancement: we could print the remote address of the client, it will help troubleshooting problems, for instance when the client shuts down the connection too early



##########
zookeeper-server/src/test/java/org/apache/zookeeper/server/admin/SnapshotCommandTest.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.zookeeper.server.admin;
+
+import static org.apache.zookeeper.server.admin.Commands.SnapshotCommand.ADMIN_SNAPSHOT_ENABLED;
+import static org.apache.zookeeper.server.admin.Commands.SnapshotCommand.ADMIN_SNAPSHOT_INTERVAL;
+import static org.apache.zookeeper.server.admin.Commands.SnapshotCommand.REQUEST_QUERY_PARAM_STREAMING;
+import static org.apache.zookeeper.server.admin.JettyAdminServerTest.URL_FORMAT;
+import static org.apache.zookeeper.server.admin.JettyAdminServerTest.jettyAdminPort;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.UnsupportedEncodingException;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.nio.file.Files;
+import java.util.HashMap;
+import java.util.Map;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.PortAssignment;
+import org.apache.zookeeper.ZKTestCase;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.common.IOUtils;
+import org.apache.zookeeper.server.ServerCnxnFactory;
+import org.apache.zookeeper.server.ZooKeeperServer;
+import org.apache.zookeeper.test.ClientBase;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.api.io.TempDir;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@TestInstance(TestInstance.Lifecycle.PER_CLASS)
+public class SnapshotCommandTest extends ZKTestCase {
+    private static final Logger LOG = LoggerFactory.getLogger(SnapshotCommandTest.class);
+
+    private static final String PATH = "/snapshot_test";
+    private static final int NODE_COUNT = 10;
+
+    private final String hostPort =  "127.0.0.1:" + PortAssignment.unique();
+    private ServerCnxnFactory cnxnFactory;
+    private JettyAdminServer adminServer;
+    private ZooKeeperServer zks;
+    private ZooKeeper zk;
+
+    @TempDir
+    static File dataDir;
+
+    @TempDir
+    static File logDir;
+
+    @BeforeAll
+    public void setup() throws Exception {
+        // start ZookeeperServer
+        System.setProperty("zookeeper.4lw.commands.whitelist", "*");

Review Comment:
   please "clear" all the system properties you set in this method in the Β΄teardown` method



-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#issuecomment-1338868682

   > One more thing: the doc mentions that this option is required for the Admin's snapshot command to work properly. If that's the case the Admin command should check whether this option is enabled, but I don't see the check there, right?
   
   Yes, you are right. Let me add the check and also the test case.


-- 
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


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

Posted by GitBox <gi...@apache.org>.
eolivelli commented on PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#issuecomment-1370663660

   I will commit this patch as soon as CI passes.
   I have triggered a new run


-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1038390377


##########
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:
   No, I don't see anyone wants to turn it off unless we run into unexpected issue.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1038417361


##########
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:
   Yeah, I think it's useful to add it to `JsonOutputter` too. Added.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#issuecomment-1335567224

   > this is a great contribution, thank you @li4wang ! Also thanks for @maoling for his previous works in this topic!
   > 
   > I think many people / projects will benefit from it. (sorry for being so slow with the reviews...)
   
   Thanks for the feedback and comments, @symat .


-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1038469137


##########
zookeeper-server/src/test/java/org/apache/zookeeper/server/util/RateLimiterTest.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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 static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import java.util.concurrent.TimeUnit;
+import org.junit.jupiter.api.Test;
+
+public class RateLimiterTest {
+
+    @Test
+    public void testAllow_withinInterval() {
+        final int rate = 2;
+        final RateLimiter rateLimiter = new RateLimiter(rate, 5, TimeUnit.SECONDS);
+        for (int i = 0; i < rate; i++) {
+            assertTrue(rateLimiter.allow());
+        }
+        assertFalse(rateLimiter.allow());
+    }
+
+    @Test
+    public void testAllow_exceedInterval() throws Exception {
+        final int interval = 1;
+
+        final RateLimiter rateLimiter = new RateLimiter(1, interval, TimeUnit.SECONDS);
+        assertTrue(rateLimiter.allow());
+        assertFalse(rateLimiter.allow());
+        Thread.sleep(TimeUnit.SECONDS.toMillis(interval) + 1);

Review Comment:
   > Please wait 2 seconds at least. Thread.sleep() in tests could easly lead to flaky test.
   changed to wait for 3 seconds
   
   
   > Additonally I think it would be nice to have a multithreaded test as well: start 10 worker in an executor to grab the numbers and verify that the 11th is disallowed within the time interval.
   
   added multithreaded test
   
   



-- 
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


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

Posted by GitBox <gi...@apache.org>.
eolivelli commented on PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#issuecomment-1340675859

   PR to unblock CI 
   https://github.com/apache/zookeeper/pull/1951


-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#issuecomment-1314060091

   thanks @eolivelli


-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1038401440


##########
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:
   This method is in `IPAuthenticationProvider `as it will be used by `IPAuthenticationProvider` when providing IP auth support for admin server APIs. 
   
   `IPAuthenticationProvider` will be enhanced for authenticating both Zookeeper client and Admin Server APIs by the client IP address.
   



-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1040657179


##########
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:
   Yep



-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#issuecomment-1339932034

   > @li4wang I have restarted CI We will commit to master branch as soon as CI passes
   > 
   > thanks!
   > 
   > looking forward for the next patch
   
   Thanks a lot @eolivelli!   I fixed the sonaytype-lift issues and re-kicked the CI. The next patch is restore command. I will submit it once this one is merged. 


-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#issuecomment-1339929737

   > +1 Thanks for taking the time.
   
   @anmolnar thanks a lot for your inputs


-- 
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


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

Posted by GitBox <gi...@apache.org>.
symat commented on PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#issuecomment-1334912500

   CI failed with unit test error:
   
   > [ERROR] Errors: 
   > [ERROR]   ConnectionMetricsTest.testRevalidateCount:65 Β» NullPointer
   
   possibly some flaky issue, I re-triggered


-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1038390377


##########
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:
   No, I don't see anyone wants to turn it off unless we run into unexpected issue, and I think it doesn't hurt to have the option available for now.



-- 
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


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

Posted by GitBox <gi...@apache.org>.
li4wang commented on code in PR #1943:
URL: https://github.com/apache/zookeeper/pull/1943#discussion_r1020629923


##########
zookeeper-server/src/main/java/org/apache/zookeeper/server/admin/StreamOutputter.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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);
+
+    public StreamOutputter() {
+    }
+
+    @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", e);

Review Comment:
   Good point. Enhanced.



-- 
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