001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.hbase.zookeeper;
020
021import java.io.IOException;
022
023import org.apache.hadoop.hbase.Abortable;
024import org.apache.hadoop.hbase.exceptions.DeserializationException;
025import org.apache.hadoop.hbase.util.Bytes;
026import org.apache.yetus.audience.InterfaceAudience;
027import org.apache.zookeeper.KeeperException;
028
029import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
030import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotCleanupProtos;
031
032/**
033 * Tracks status of snapshot auto cleanup based on TTL
034 */
035@InterfaceAudience.Private
036public class SnapshotCleanupTracker extends ZKNodeTracker {
037
038  /**
039   * Constructs a new ZK node tracker.
040   *
041   * <p>After construction, use {@link #start} to kick off tracking.
042   *
043   * @param watcher reference to the {@link ZKWatcher} which also contains configuration and
044   *   constants
045   * @param abortable used to abort if a fatal error occurs
046   */
047  public SnapshotCleanupTracker(ZKWatcher watcher, Abortable abortable) {
048    super(watcher, watcher.getZNodePaths().snapshotCleanupZNode, abortable);
049  }
050
051  /**
052   * Returns the current state of the snapshot auto cleanup based on TTL
053   *
054   * @return <code>true</code> if the snapshot auto cleanup is enabled,
055   *   <code>false</code> otherwise.
056   */
057  public boolean isSnapshotCleanupEnabled() {
058    byte[] snapshotCleanupZNodeData = super.getData(false);
059    try {
060      // if data in ZK is null, use default of on.
061      return snapshotCleanupZNodeData == null ||
062          parseFrom(snapshotCleanupZNodeData).getSnapshotCleanupEnabled();
063    } catch (DeserializationException dex) {
064      LOG.error("ZK state for Snapshot Cleanup could not be parsed " +
065          Bytes.toStringBinary(snapshotCleanupZNodeData), dex);
066      // return false to be safe.
067      return false;
068    }
069  }
070
071  /**
072   * Set snapshot auto clean on/off
073   *
074   * @param snapshotCleanupEnabled true if the snapshot auto cleanup should be on,
075   *   false otherwise
076   * @throws KeeperException if ZooKeeper operation fails
077   */
078  public void setSnapshotCleanupEnabled(final boolean snapshotCleanupEnabled)
079      throws KeeperException {
080    byte [] snapshotCleanupZNodeData = toByteArray(snapshotCleanupEnabled);
081    try {
082      ZKUtil.setData(watcher, watcher.getZNodePaths().snapshotCleanupZNode,
083          snapshotCleanupZNodeData);
084    } catch(KeeperException.NoNodeException nne) {
085      ZKUtil.createAndWatch(watcher, watcher.getZNodePaths().snapshotCleanupZNode,
086          snapshotCleanupZNodeData);
087    }
088    super.nodeDataChanged(watcher.getZNodePaths().snapshotCleanupZNode);
089  }
090
091  private byte[] toByteArray(final boolean isSnapshotCleanupEnabled) {
092    SnapshotCleanupProtos.SnapshotCleanupState.Builder builder =
093        SnapshotCleanupProtos.SnapshotCleanupState.newBuilder();
094    builder.setSnapshotCleanupEnabled(isSnapshotCleanupEnabled);
095    return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
096  }
097
098  private SnapshotCleanupProtos.SnapshotCleanupState parseFrom(final byte[] pbBytes)
099      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}