[GitHub] zookeeper pull request #180: ZOOKEEPER-2700 add JMX `takeSnapshot` method an...
Github user flier commented on a diff in the pull request: https://github.com/apache/zookeeper/pull/180#discussion_r104831719 --- Diff: src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java --- @@ -303,15 +305,38 @@ public void loadData() throws IOException, InterruptedException { public void takeSnapshot(){ --- End diff -- @hanm sure, please check the [zkpipe](https://github.com/flier/zkpipe) project, we are using it to tracing and audit zookeeper operations --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] zookeeper pull request #180: ZOOKEEPER-2700 add JMX `takeSnapshot` method an...
Github user flier commented on a diff in the pull request: https://github.com/apache/zookeeper/pull/180#discussion_r102372975 --- Diff: src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java --- @@ -303,15 +305,38 @@ public void loadData() throws IOException, InterruptedException { public void takeSnapshot(){ --- End diff -- @hanm Thanks for your advice :) We are using a similar structure as `2.2 Alternative 2 â Learners` in the paper, it is good enough for most of online scenes. I don't think it is worthy to introduce another layer because we give up the write operation to all observers, just use it as a read only view. For the remote replication, I doubt it also blocked by packet loss rate like Observer. On the other hand, we have an internal project named `zkpipe`, it read Zookeeper snapshot/binlog and send it to a Kafka topic, our client could choose to rebuild the transaction or subscribe the changes. I believe it will better than hack Zookeeper itself. If you have interested, I could push it to github later. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] zookeeper pull request #180: ZOOKEEPER-2700 add JMX `takeSnapshot` method an...
Github user flier commented on a diff in the pull request: https://github.com/apache/zookeeper/pull/180#discussion_r102271596 --- Diff: src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java --- @@ -303,15 +305,38 @@ public void loadData() throws IOException, InterruptedException { public void takeSnapshot(){ --- End diff -- @revans2 It's not easy to find `SyncRequestProcessor `, but we may use [reversed workaroud](https://github.com/apache/zookeeper/pull/180/files#diff-a8615898f7be77e9cfdb686e21c97b1eL123), let's `SyncRequestProcessor` use `zks.getLastProcessedZxid()-zks.getLastSnapshotZxid()` to trace snapshot --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] zookeeper issue #180: ZOOKEEPER-2700 add JMX `takeSnapshot` method and Jetty...
Github user flier commented on the issue: https://github.com/apache/zookeeper/pull/180 @revans2 added a [generating](https://github.com/apache/zookeeper/pull/180/files#diff-8c1d5f61ac728fda8f4b706ae4277364R524) field to the response of `TakeSnapshotCommand ` command, and JMX `takeSnapshot` will return a boolean for same reason. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] zookeeper pull request #180: ZOOKEEPER-2700 add JMX `takeSnapshot` method an...
Github user flier commented on a diff in the pull request: https://github.com/apache/zookeeper/pull/180#discussion_r102263924 --- Diff: src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java --- @@ -303,15 +305,38 @@ public void loadData() throws IOException, InterruptedException { public void takeSnapshot(){ --- End diff -- I have some scenes need to take snapshot, for example First, our major Zookeeper cluster was deployed in an AWS zone, some observers running at a dozen IDC. We use this topological structure because Zookeeper cluster is not friendly to multi-IDC deployment. Besize, our zookeeper snapshot and transaction logs are huge, because some wrong client usage that hard to fix in short time :( Sometimes, we plan to maintains the major cluster, we have to start another mirror cluster in same DC, and switch from the major cluster to the mirror cluster. If we do it faster enough, the observer and client will not concern the changes. That's why we need take snapshot to speed up the migration. If something got wrong, we could switch back to the old cluster, lost some transaction better than the whole system down. Second, our backup policy need a daily/hourly offline backup, to AWS S3 or other DC. I would like to take and upload a latest and clean snapshot, instead of tar an old snapshot with a number of transaction logs. Third, sometimes we need to deploy a new observer or a testing cluster in different DC, we have to copy the latest snapshot offline, because Zookeeper observer sync progress may become very slow, the TCP window could drop to 10-20KB/s in the 40-60% packet loss rate. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] zookeeper pull request #180: ZOOKEEPER-2700 add JMX `takeSnapshot` method an...
Github user flier commented on a diff in the pull request: https://github.com/apache/zookeeper/pull/180#discussion_r102238878 --- Diff: src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java --- @@ -303,15 +305,38 @@ public void loadData() throws IOException, InterruptedException { public void takeSnapshot(){ --- End diff -- If we are not expect to take snapshot on same time, the most easy way is to use `AtomicBoolean ` protect the `takeSnapshot` from all the code path, which may block the `SyncRequestProcessor` a while if a manual task is ongoing. My current code is assuming the background `SyncRequestProcessor` have higher priority. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] zookeeper pull request #180: ZOOKEEPER-2700 add JMX `takeSnapshot` method an...
Github user flier commented on a diff in the pull request: https://github.com/apache/zookeeper/pull/180#discussion_r102028811 --- Diff: src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java --- @@ -303,15 +305,38 @@ public void loadData() throws IOException, InterruptedException { public void takeSnapshot(){ try { +lastSnapshotZxid = zkDb.getDataTreeLastProcessedZxid(); +isGeneratingSnapshot.incrementAndGet(); + txnLogFactory.save(zkDb.getDataTree(), zkDb.getSessionWithTimeOuts()); } catch (IOException e) { LOG.error("Severe unrecoverable error, exiting", e); // This is a severe error that we cannot recover from, // so we need to exit System.exit(10); +} finally { +isGeneratingSnapshot.decrementAndGet(); } } +public boolean tryTakeSnapshot() { --- End diff -- hmm...I haven't working on Java for years, forgive me :) --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] zookeeper pull request #180: ZOOKEEPER-2700 add JMX `takeSnapshot` method an...
Github user flier commented on a diff in the pull request: https://github.com/apache/zookeeper/pull/180#discussion_r102028465 --- Diff: src/java/main/org/apache/zookeeper/server/ZooKeeperServer.java --- @@ -126,6 +125,9 @@ private final ZooKeeperServerListener listener; private ZooKeeperServerShutdownHandler zkShutdownHandler; +private volatile long lastSnapshotZxid; +private AtomicInteger isGeneratingSnapshot = new AtomicInteger(0); --- End diff -- In fact, I'm not sure whether Zookeeper backend threads will take snapshot in parallel. So, I choose to use a `AtomicInteger` to protect manual call `takeSnapshot`. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] zookeeper issue #180: ZOOKEEPER-2700 add JMX `takeSnapshot` method and Jetty...
Github user flier commented on the issue: https://github.com/apache/zookeeper/pull/180 According the review comments and [ZOOKEEPER-1729](https://issues.apache.org/jira/browse/ZOOKEEPER-1729), I have submited another patch to add JMX `takeSnapshot` method and Jetty Admin `snap` command to take snapshot, with test cases --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] zookeeper pull request #180: ZOOKEEPER-2700 add `snap` command to take snaps...
Github user flier commented on a diff in the pull request: https://github.com/apache/zookeeper/pull/180#discussion_r101880473 --- Diff: src/java/main/org/apache/zookeeper/server/command/SnapCommand.java --- @@ -0,0 +1,53 @@ +/** + * 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.command; + +import org.apache.zookeeper.server.ServerCnxn; +import org.apache.zookeeper.server.ZooKeeperThread; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.PrintWriter; + +public class SnapCommand extends AbstractFourLetterCommand { +private static final Logger LOG = LoggerFactory.getLogger(SnapCommand.class); + +SnapCommand(PrintWriter pw, ServerCnxn serverCnxn) { +super(pw,serverCnxn); +} + +@Override +public void commandRun() throws IOException { +if (!isZKServerRunning()) { +pw.println(ZK_NOT_SERVING); +} else { +Thread snapInProcess = new ZooKeeperThread("Snapshot Thread") { +public void run() { +try { +zkServer.takeSnapshot(); --- End diff -- I have wrapped the manual requests from `snap` command or JMX with a `tryTakeSnapshot` method, which will skip action when there are busy. If you believe it is necessary, we could merge the check into `takeSnapshot`, which may impact all the server workflow. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] zookeeper pull request #180: ZOOKEEPER-2700 add `snap` command to take snaps...
Github user flier commented on a diff in the pull request: https://github.com/apache/zookeeper/pull/180#discussion_r101880274 --- Diff: src/java/main/org/apache/zookeeper/server/command/SnapCommand.java --- @@ -0,0 +1,53 @@ +/** + * 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.command; + +import org.apache.zookeeper.server.ServerCnxn; +import org.apache.zookeeper.server.ZooKeeperThread; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.PrintWriter; + +public class SnapCommand extends AbstractFourLetterCommand { +private static final Logger LOG = LoggerFactory.getLogger(SnapCommand.class); + +SnapCommand(PrintWriter pw, ServerCnxn serverCnxn) { +super(pw,serverCnxn); +} + +@Override +public void commandRun() throws IOException { +if (!isZKServerRunning()) { +pw.println(ZK_NOT_SERVING); +} else { +Thread snapInProcess = new ZooKeeperThread("Snapshot Thread") { --- End diff -- @revans2 now, only generate snapshot when idle and last zxid changed. @hanm I think `snap` should be a command to impact only one instance, no an administration task to all instance in cluster. @eribeiro added a `takeSnapshot` method to JMX --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] zookeeper pull request #180: ZOOKEEPER-2700 add command to take snapshot
GitHub user flier opened a pull request: https://github.com/apache/zookeeper/pull/180 ZOOKEEPER-2700 add command to take snapshot When cold backup or remote offline sync Zookeeper instances, we need the latest snapshot. Add a four letter `snap` command to force Zookeeper to generate snapshot. You can merge this pull request into a Git repository by running: $ git pull https://github.com/flier/zookeeper ZOOKEEPER-2700 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/zookeeper/pull/180.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #180 commit e1dc4c94ea41a3fc0e5511fe1d69e02f27f4f0b8 Author: Flier Lu Date: 2017-02-17T13:48:19Z add command to take snapshot commit 77402134bf172566b69b0ffe72f19f18b93ec989 Author: Flier Lu Date: 2017-02-17T13:57:51Z add document --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[jira] [Created] (ZOOKEEPER-2700) Force ZooKeeper to generate snapshot
Flier Lu created ZOOKEEPER-2700: --- Summary: Force ZooKeeper to generate snapshot Key: ZOOKEEPER-2700 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2700 Project: ZooKeeper Issue Type: Improvement Reporter: Flier Lu Priority: Minor When cold backup or remote offline sync Zookeeper instances, we need the latest snapshot. Add a four letter `snap` command to force Zookeeper to generate snapshot. -- This message was sent by Atlassian JIRA (v6.3.15#6346)