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.master;
019
020import java.io.IOException;
021import org.apache.hadoop.hbase.exceptions.DeserializationException;
022import org.apache.hadoop.hbase.master.region.MasterRegion;
023import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
024import org.apache.yetus.audience.InterfaceAudience;
025import org.apache.zookeeper.KeeperException;
026
027/**
028 * Store a boolean state.
029 */
030@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "UG_SYNC_SET_UNSYNC_GET",
031    justification = "the flag is volatile")
032@InterfaceAudience.Private
033public abstract class BooleanStateStore extends MasterStateStore {
034
035  private volatile boolean on;
036
037  protected BooleanStateStore(MasterRegion masterRegion, String stateName, ZKWatcher watcher,
038    String zkPath) throws IOException, KeeperException, DeserializationException {
039    super(masterRegion, stateName, watcher, zkPath);
040    byte[] state = getState();
041    this.on = state == null || parseFrom(state);
042  }
043
044  /**
045   * Returns true if the flag is on, otherwise false
046   */
047  public boolean get() {
048    return on;
049  }
050
051  /**
052   * Set the flag on/off.
053   * @param on true if the flag should be on, false otherwise
054   * @throws IOException if the operation fails
055   * @return returns the previous state
056   */
057  public synchronized boolean set(boolean on) throws IOException {
058    byte[] state = toByteArray(on);
059    setState(state);
060    boolean prevOn = this.on;
061    this.on = on;
062    return prevOn;
063  }
064
065  protected abstract byte[] toByteArray(boolean on);
066
067  protected abstract boolean parseFrom(byte[] bytes) throws DeserializationException;
068}