1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.zookeeper;
20
21 import java.io.IOException;
22
23 import org.apache.hadoop.hbase.Abortable;
24 import org.apache.hadoop.hbase.classification.InterfaceAudience;
25 import org.apache.hadoop.hbase.exceptions.DeserializationException;
26 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
27 import org.apache.hadoop.hbase.protobuf.generated.SnapshotCleanupProtos;
28 import org.apache.hadoop.hbase.util.Bytes;
29 import org.apache.zookeeper.KeeperException;
30
31
32
33
34
35 @InterfaceAudience.Private
36 public class SnapshotCleanupTracker extends ZooKeeperNodeTracker {
37
38
39
40
41
42
43
44
45
46
47 public SnapshotCleanupTracker(ZooKeeperWatcher watcher, Abortable abortable) {
48 super(watcher, watcher.snapshotCleanupZNode, abortable);
49 }
50
51
52
53
54
55
56
57 public boolean isSnapshotCleanupEnabled() {
58 byte[] snapshotCleanupZNodeData = super.getData(false);
59 try {
60
61 return snapshotCleanupZNodeData == null ||
62 parseFrom(snapshotCleanupZNodeData).getSnapshotCleanupEnabled();
63 } catch (DeserializationException dex) {
64 LOG.error("ZK state for Snapshot Cleanup could not be parsed " +
65 Bytes.toStringBinary(snapshotCleanupZNodeData), dex);
66
67 return false;
68 }
69 }
70
71
72
73
74
75
76
77
78 public void setSnapshotCleanupEnabled(final boolean snapshotCleanupEnabled)
79 throws KeeperException {
80 byte [] snapshotCleanupZNodeData = toByteArray(snapshotCleanupEnabled);
81 try {
82 ZKUtil.setData(watcher, watcher.snapshotCleanupZNode,
83 snapshotCleanupZNodeData);
84 } catch(KeeperException.NoNodeException nne) {
85 ZKUtil.createAndWatch(watcher, watcher.snapshotCleanupZNode,
86 snapshotCleanupZNodeData);
87 }
88 super.nodeDataChanged(watcher.snapshotCleanupZNode);
89 }
90
91 private byte[] toByteArray(final boolean isSnapshotCleanupEnabled) {
92 SnapshotCleanupProtos.SnapshotCleanupState.Builder builder =
93 SnapshotCleanupProtos.SnapshotCleanupState.newBuilder();
94 builder.setSnapshotCleanupEnabled(isSnapshotCleanupEnabled);
95 return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
96 }
97
98 private SnapshotCleanupProtos.SnapshotCleanupState parseFrom(final byte[] pbBytes)
99 throws DeserializationException {
100 ProtobufUtil.expectPBMagicPrefix(pbBytes);
101 SnapshotCleanupProtos.SnapshotCleanupState.Builder builder =
102 SnapshotCleanupProtos.SnapshotCleanupState.newBuilder();
103 try {
104 int magicLen = ProtobufUtil.lengthOfPBMagic();
105 ProtobufUtil.mergeFrom(builder, pbBytes, magicLen, pbBytes.length - magicLen);
106 } catch (IOException e) {
107 throw new DeserializationException(e);
108 }
109 return builder.build();
110 }
111
112 }