Skip to content

Commit c1b9374

Browse files
authored
Update metric values in backup manager (#31)
This PR is second commit of zk backup monitoring implementation. This commit integrates BackupStats with BackupManager.
1 parent 215741e commit c1b9374

File tree

5 files changed

+190
-6
lines changed

5 files changed

+190
-6
lines changed

zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/BackupManager.java

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818

1919
package org.apache.zookeeper.server.backup;
2020

21+
import com.google.common.annotations.VisibleForTesting;
2122
import com.google.common.base.Preconditions;
2223
import org.apache.commons.lang.NullArgumentException;
2324
import org.apache.commons.lang.time.StopWatch;
@@ -63,7 +64,8 @@ public class BackupManager {
6364
private final BackupStorageProvider backupStorage;
6465
private final long serverId;
6566
private final String namespace;
66-
private BackupBean backupBean = null;
67+
@VisibleForTesting
68+
protected BackupBean backupBean = null;
6769
private BackupStats backupStats = null;
6870

6971
/**
@@ -366,9 +368,11 @@ protected void startIteration() {
366368
// for the current iteration so we don't keep chasing our own tail as
367369
// new transactions get written.
368370
iterationEndPoint = snapLog.getLastLoggedZxid();
371+
backupStats.setTxnLogBackupIterationStart();
369372
}
370373

371374
protected void endIteration(boolean errorFree) {
375+
backupStats.setTxnLogBackupIterationDone(errorFree);
372376
iterationEndPoint = 0L;
373377
}
374378

@@ -551,6 +555,7 @@ protected void initialize() throws IOException {
551555
* @throws IOException
552556
*/
553557
protected void startIteration() throws IOException {
558+
backupStats.setSnapshotBackupIterationStart();
554559
filesToBackup.clear();
555560

556561
// Get all available snapshots excluding the ones whose lastProcessedZxid falls into the
@@ -604,6 +609,7 @@ protected void startIteration() throws IOException {
604609
}
605610

606611
protected void endIteration(boolean errorFree) {
612+
backupStats.setSnapshotBackupIterationDone(errorFree);
607613
filesToBackup.clear();
608614
}
609615

@@ -620,6 +626,7 @@ protected void backupComplete(BackupFile file) throws IOException {
620626
backedupSnapZxid = file.getMinZxid();
621627
backupStatus.update(backedupLogZxid, backedupSnapZxid);
622628
}
629+
backupStats.incrementNumberOfSnapshotFilesBackedUpThisIteration();
623630

624631
logger.info("Updated backedup snap zxid to {}", ZxidUtils.zxidToString(backedupSnapZxid));
625632
}

zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/monitoring/BackupBean.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -71,7 +71,7 @@ public long getLastSnapshotIterationDuration() {
7171
}
7272

7373
@Override
74-
public long getNumberOfSnapshotFilesBackedUpLastIteration() {
74+
public int getNumberOfSnapshotFilesBackedUpLastIteration() {
7575
return backupStats.getNumberOfSnapshotFilesBackedUpLastIteration();
7676
}
7777

zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/monitoring/BackupMXBean.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -56,7 +56,7 @@ public interface BackupMXBean {
5656
* Gauge
5757
* @return Number of snapshot files that were backed up to backup storage in last snapshot backup iteration
5858
*/
59-
long getNumberOfSnapshotFilesBackedUpLastIteration();
59+
int getNumberOfSnapshotFilesBackedUpLastIteration();
6060

6161
// Transaction log backup metrics
6262

zookeeper-server/src/main/java/org/apache/zookeeper/server/backup/monitoring/BackupStats.java

Lines changed: 64 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -32,10 +32,14 @@ public class BackupStats {
3232
private boolean snapshotBackupActive = false;
3333
private long snapshotIterationDuration = 0L;
3434
private int numberOfSnapshotFilesBackedUpLastIteration = 0;
35+
private int numberOfSnapshotFilesBackedUpThisIteration = 0;
36+
private long lastSnapshotBackupIterationStartTime = System.currentTimeMillis();
37+
3538
private int failedTxnLogIterationCount = 0;
3639
private long lastSuccessfulTxnLogBackupIterationFinishTime = System.currentTimeMillis();
3740
private boolean txnLogBackupActive = false;
3841
private long txnLogIterationDuration = 0L;
42+
private long lastTxnLogBackupIterationStartTime = System.currentTimeMillis();
3943

4044
// Snapshot backup metrics
4145

@@ -78,10 +82,45 @@ public long getSnapshotIterationDuration() {
7882
* Gauge
7983
* @return Number of snapshot files that were backed up to backup storage in last snapshot backup iteration
8084
*/
81-
public long getNumberOfSnapshotFilesBackedUpLastIteration() {
85+
public int getNumberOfSnapshotFilesBackedUpLastIteration() {
8286
return numberOfSnapshotFilesBackedUpLastIteration;
8387
}
8488

89+
/**
90+
* Record the status and timestamp when a snapshot backup iteration starts
91+
*/
92+
public void setSnapshotBackupIterationStart() {
93+
lastSnapshotBackupIterationStartTime = System.currentTimeMillis();
94+
snapshotBackupActive = true;
95+
numberOfSnapshotFilesBackedUpThisIteration = 0;
96+
}
97+
98+
/**
99+
* Record the status and timestamp when a snapshot backup iteration finishes
100+
* @param errorFree If this iteration finishes without error
101+
*/
102+
public void setSnapshotBackupIterationDone(boolean errorFree) {
103+
long finishTime = System.currentTimeMillis();
104+
snapshotIterationDuration = finishTime - lastSnapshotBackupIterationStartTime;
105+
snapshotBackupActive = false;
106+
numberOfSnapshotFilesBackedUpLastIteration = numberOfSnapshotFilesBackedUpThisIteration;
107+
numberOfSnapshotFilesBackedUpThisIteration = 0;
108+
if (errorFree) {
109+
lastSuccessfulSnapshotBackupIterationFinishTime = finishTime;
110+
failedSnapshotIterationCount = 0;
111+
} else {
112+
failedSnapshotIterationCount++;
113+
}
114+
}
115+
116+
/**
117+
* To be called during snapshot backup iteration every time one more snapshot file is backed up
118+
* A helper to keep track value for "numberOfSnapshotFilesBackedUpLastIteration" metric
119+
*/
120+
public void incrementNumberOfSnapshotFilesBackedUpThisIteration() {
121+
numberOfSnapshotFilesBackedUpThisIteration++;
122+
}
123+
85124
// Transaction log backup metrics
86125

87126
/**
@@ -118,4 +157,28 @@ public boolean getTxnLogBackupActiveStatus() {
118157
public long getTxnLogIterationDuration() {
119158
return txnLogIterationDuration;
120159
}
160+
161+
/**
162+
* Record the status and timestamp when a txn log backup iteration starts
163+
*/
164+
public void setTxnLogBackupIterationStart() {
165+
lastTxnLogBackupIterationStartTime = System.currentTimeMillis();
166+
txnLogBackupActive = true;
167+
}
168+
169+
/**
170+
* Record the status and timestamp when a txn log backup iteration finishes
171+
* @param errorFree If this iteration finishes without error
172+
*/
173+
public void setTxnLogBackupIterationDone(boolean errorFree) {
174+
long finishTime = System.currentTimeMillis();
175+
txnLogBackupActive = false;
176+
txnLogIterationDuration = finishTime - lastTxnLogBackupIterationStartTime;
177+
if (errorFree) {
178+
lastSuccessfulTxnLogBackupIterationFinishTime = finishTime;
179+
failedTxnLogIterationCount = 0;
180+
} else {
181+
failedTxnLogIterationCount++;
182+
}
183+
}
121184
}

zookeeper-server/src/test/java/org/apache/zookeeper/server/backup/BackupBeanTest.java

Lines changed: 116 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -23,16 +23,19 @@
2323
import java.util.Optional;
2424
import java.util.Set;
2525

26+
import org.apache.zookeeper.CreateMode;
2627
import org.apache.zookeeper.DummyWatcher;
2728
import org.apache.zookeeper.PortAssignment;
2829
import org.apache.zookeeper.ZKTestCase;
30+
import org.apache.zookeeper.ZooDefs;
2931
import org.apache.zookeeper.ZooKeeper;
3032
import org.apache.zookeeper.jmx.MBeanRegistry;
3133
import org.apache.zookeeper.jmx.ZKMBeanInfo;
3234
import org.apache.zookeeper.server.ServerCnxn;
3335
import org.apache.zookeeper.server.ServerCnxnFactory;
3436
import org.apache.zookeeper.server.SyncRequestProcessor;
3537
import org.apache.zookeeper.server.ZooKeeperServer;
38+
import org.apache.zookeeper.server.backup.monitoring.BackupBean;
3639
import org.apache.zookeeper.server.backup.storage.BackupStorageProvider;
3740
import org.apache.zookeeper.server.backup.storage.impl.FileSystemBackupStorage;
3841
import org.apache.zookeeper.server.persistence.FileTxnSnapLog;
@@ -58,7 +61,6 @@ public class BackupBeanTest extends ZKTestCase {
5861
private ZooKeeperServer zks;
5962
private ServerCnxnFactory serverCnxnFactory;
6063
private BackupStorageProvider backupStorage;
61-
private BackupManager backupManager;
6264
private BackupStatus backupStatus;
6365
private FileTxnSnapLog snapLog;
6466
private BackupConfig backupConfig;
@@ -128,7 +130,6 @@ public void teardown() throws Exception {
128130
Assert.assertTrue("waiting for server to shutdown",
129131
ClientBase.waitForServerDown(HOSTPORT, CONNECTION_TIMEOUT));
130132

131-
backupManager = null;
132133
backupStatus = null;
133134
serverCnxnFactory = null;
134135
zks = null;
@@ -157,4 +158,117 @@ private boolean containsMBean(Set<ZKMBeanInfo> mbeanSet, String mbeanName, boole
157158
.findAny();
158159
return foundMBean.isPresent();
159160
}
161+
162+
@Test
163+
public void testMBeanUpdate() throws Exception {
164+
MockBackupManager backupManager = new MockBackupManager(dataDir, dataDir, dataDir, backupTmpDir, 15,
165+
new FileSystemBackupStorage(backupConfig), TEST_NAMESPACE, 0);
166+
BackupBean backupBean = backupManager.getBackupBean();
167+
168+
Assert.assertEquals(0, backupBean.getMinutesSinceLastSuccessfulSnapshotIteration());
169+
Assert.assertEquals(0, backupBean.getMinutesSinceLastSuccessfulTxnLogIteration());
170+
171+
String[] nodeNames = {"/firstNode", "/secondNode", "/thirdNode", "/fourthNode", "/fifthNode"};
172+
173+
createNode(connection, nodeNames[0]);
174+
backupManager.getSnapBackup(snapLog, true).run(1);
175+
backupManager.getLogBackup(snapLog, true).run(1);
176+
Assert.assertTrue(backupBean.getLastSnapshotIterationDuration() > 0L);
177+
Assert.assertTrue(backupBean.getLastTxnLogIterationDuration() > 0L);
178+
Assert.assertEquals(1, backupBean.getNumberOfSnapshotFilesBackedUpLastIteration());
179+
Assert.assertEquals(0, backupBean.getNumConsecutiveFailedSnapshotIterations());
180+
Assert.assertEquals(0, backupBean.getNumConsecutiveFailedTxnLogIterations());
181+
Assert.assertFalse(backupBean.getSnapshotBackupActiveStatus());
182+
Assert.assertFalse(backupBean.getTxnLogBackupActiveStatus());
183+
184+
createNode(connection, nodeNames[1]);
185+
backupManager.getSnapBackup(snapLog, false).run(1);
186+
backupManager.getLogBackup(snapLog, true).run(1);
187+
Assert.assertEquals(1, backupBean.getNumConsecutiveFailedSnapshotIterations());
188+
Assert.assertEquals(0, backupBean.getNumConsecutiveFailedTxnLogIterations());
189+
190+
createNode(connection, nodeNames[2]);
191+
backupManager.getSnapBackup(snapLog, true).run(1);
192+
backupManager.getLogBackup(snapLog, false).run(1);
193+
Assert.assertEquals(0, backupBean.getNumConsecutiveFailedSnapshotIterations());
194+
Assert.assertEquals(1, backupBean.getNumConsecutiveFailedTxnLogIterations());
195+
196+
createNode(connection, nodeNames[3]);
197+
backupManager.getSnapBackup(snapLog, false).run(1);
198+
backupManager.getLogBackup(snapLog, false).run(1);
199+
Assert.assertEquals(1, backupBean.getNumConsecutiveFailedSnapshotIterations());
200+
Assert.assertEquals(2, backupBean.getNumConsecutiveFailedTxnLogIterations());
201+
202+
createNode(connection, nodeNames[4]);
203+
backupManager.getSnapBackup(snapLog, true).run(1);
204+
backupManager.getLogBackup(snapLog, true).run(1);
205+
Assert.assertEquals(0, backupBean.getNumConsecutiveFailedSnapshotIterations());
206+
Assert.assertEquals(0, backupBean.getNumConsecutiveFailedTxnLogIterations());
207+
208+
Thread.sleep(60 * 1000);
209+
Assert.assertTrue(backupBean.getMinutesSinceLastSuccessfulSnapshotIteration() > 0L);
210+
Assert.assertTrue(backupBean.getMinutesSinceLastSuccessfulTxnLogIteration() > 0L);
211+
}
212+
213+
private void createNode(ZooKeeper zk, String path) throws Exception {
214+
zk.create(path, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
215+
}
216+
217+
private static class MockBackupManager extends BackupManager {
218+
219+
public MockBackupManager(File snapDir, File dataLogDir, File backupStatusDir, File tmpDir,
220+
int backupIntervalInMinutes, BackupStorageProvider backupStorageProvider, String namespace,
221+
long serverId) throws IOException {
222+
super(snapDir, dataLogDir, backupStatusDir, tmpDir, backupIntervalInMinutes,
223+
backupStorageProvider, namespace, serverId);
224+
}
225+
226+
public BackupBean getBackupBean() {
227+
return this.backupBean;
228+
}
229+
230+
public MockSnapshotBackupProcess getSnapBackup(FileTxnSnapLog snapLog, boolean errorFree) {
231+
return new MockSnapshotBackupProcess(snapLog, errorFree, backupBean);
232+
}
233+
234+
public MockTxnLogBackupProcess getLogBackup(FileTxnSnapLog snapLog, boolean errorFree) {
235+
return new MockTxnLogBackupProcess(snapLog, errorFree, backupBean);
236+
}
237+
238+
private class MockSnapshotBackupProcess extends BackupManager.SnapBackup {
239+
private boolean expectedErrorFree;
240+
private BackupBean backupBean;
241+
242+
public MockSnapshotBackupProcess(FileTxnSnapLog snapLog, boolean expectedErrorFree,
243+
BackupBean backupBean) {
244+
super(snapLog);
245+
this.expectedErrorFree = expectedErrorFree;
246+
this.backupBean = backupBean;
247+
}
248+
249+
@Override
250+
protected void endIteration(boolean errorFree) {
251+
Assert.assertTrue(backupBean.getSnapshotBackupActiveStatus());
252+
super.endIteration(expectedErrorFree);
253+
}
254+
}
255+
256+
private class MockTxnLogBackupProcess extends BackupManager.TxnLogBackup {
257+
private boolean expectedErrorFree;
258+
private BackupBean backupBean;
259+
260+
public MockTxnLogBackupProcess(FileTxnSnapLog snapLog, boolean expectedErrorFree,
261+
BackupBean backupBean) {
262+
super(snapLog);
263+
this.expectedErrorFree = expectedErrorFree;
264+
this.backupBean = backupBean;
265+
}
266+
267+
@Override
268+
protected void endIteration(boolean errorFree) {
269+
Assert.assertTrue(backupBean.getTxnLogBackupActiveStatus());
270+
super.endIteration(expectedErrorFree);
271+
}
272+
}
273+
}
160274
}

0 commit comments

Comments
 (0)