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 */
018package org.apache.hadoop.hbase.zookeeper;
019
020import java.io.IOException;
021import org.apache.hadoop.hbase.Abortable;
022import org.apache.hadoop.hbase.exceptions.DeserializationException;
023import org.apache.hadoop.hbase.util.Bytes;
024import org.apache.yetus.audience.InterfaceAudience;
025import org.apache.zookeeper.KeeperException;
026
027import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
028import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotCleanupProtos;
029
030/**
031 * Tracks status of snapshot auto cleanup based on TTL
032 */
033@InterfaceAudience.Private
034public class SnapshotCleanupTracker extends ZKNodeTracker {
035
036  /**
037   * Constructs a new ZK node tracker.
038   * <p>
039   * After construction, use {@link #start} to kick off tracking.
040   * @param watcher   reference to the {@link ZKWatcher} which also contains configuration and
041   *                  constants
042   * @param abortable used to abort if a fatal error occurs
043   */
044  public SnapshotCleanupTracker(ZKWatcher watcher, Abortable abortable) {
045    super(watcher, watcher.getZNodePaths().snapshotCleanupZNode, abortable);
046  }
047
048  /**
049   * Returns the current state of the snapshot auto cleanup based on TTL
050   * @return <code>true</code> if the snapshot auto cleanup is enabled, <code>false</code>
051   *         otherwise.
052   */
053  public boolean isSnapshotCleanupEnabled() {
054    byte[] snapshotCleanupZNodeData = super.getData(false);
055    try {
056      // if data in ZK is null, use default of on.
057      return snapshotCleanupZNodeData == null
058        || parseFrom(snapshotCleanupZNodeData).getSnapshotCleanupEnabled();
059    } catch (DeserializationException dex) {
060      LOG.error("ZK state for Snapshot Cleanup could not be parsed "
061        + Bytes.toStringBinary(snapshotCleanupZNodeData), dex);
062      // return false to be safe.
063      return false;
064    }
065  }
066
067  /**
068   * Set snapshot auto clean on/off
069   * @param snapshotCleanupEnabled true if the snapshot auto cleanup should be on, false otherwise
070   * @throws KeeperException if ZooKeeper operation fails
071   */
072  public void setSnapshotCleanupEnabled(final boolean snapshotCleanupEnabled)
073    throws KeeperException {
074    byte[] snapshotCleanupZNodeData = toByteArray(snapshotCleanupEnabled);
075    try {
076      ZKUtil.setData(watcher, watcher.getZNodePaths().snapshotCleanupZNode,
077        snapshotCleanupZNodeData);
078    } catch (KeeperException.NoNodeException nne) {
079      ZKUtil.createAndWatch(watcher, watcher.getZNodePaths().snapshotCleanupZNode,
080        snapshotCleanupZNodeData);
081    }
082    super.nodeDataChanged(watcher.getZNodePaths().snapshotCleanupZNode);
083  }
084
085  private byte[] toByteArray(final boolean isSnapshotCleanupEnabled) {
086    SnapshotCleanupProtos.SnapshotCleanupState.Builder builder =
087      SnapshotCleanupProtos.SnapshotCleanupState.newBuilder();
088    builder.setSnapshotCleanupEnabled(isSnapshotCleanupEnabled);
089    return ProtobufUtil.prependPBMagic(builder.build().toByteArray());
090  }
091
092  private SnapshotCleanupProtos.SnapshotCleanupState parseFrom(final byte[] pbBytes)
093    throws DeserializationException {
094    ProtobufUtil.expectPBMagicPrefix(pbBytes);
095    SnapshotCleanupProtos.SnapshotCleanupState.Builder builder =
096      SnapshotCleanupProtos.SnapshotCleanupState.newBuilder();
097    try {
098      int magicLen = ProtobufUtil.lengthOfPBMagic();
099      ProtobufUtil.mergeFrom(builder, pbBytes, magicLen, pbBytes.length - magicLen);
100    } catch (IOException e) {
101      throw new DeserializationException(e);
102    }
103    return builder.build();
104  }
105
106}