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 static org.apache.hadoop.hbase.master.region.MasterRegionFactory.STATE_FAMILY;
021
022import java.io.IOException;
023import org.apache.hadoop.hbase.client.Get;
024import org.apache.hadoop.hbase.client.Put;
025import org.apache.hadoop.hbase.client.Result;
026import org.apache.hadoop.hbase.master.region.MasterRegion;
027import org.apache.hadoop.hbase.util.Bytes;
028import org.apache.hadoop.hbase.zookeeper.ZKUtil;
029import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
030import org.apache.yetus.audience.InterfaceAudience;
031import org.apache.zookeeper.KeeperException;
032import org.slf4j.Logger;
033import org.slf4j.LoggerFactory;
034
035/**
036 * A state storage which stores the state in master local region.
037 * <p/>
038 * We used to store some persistent state on zookeeper, so here we provide the ability to migrate
039 * the state from zookeeper.
040 * <p/>
041 * Since parsing the state may introduce some extra overhead, we make this class abstract and the
042 * get and set state methods protected. Sub classes should store their state in decoded format to
043 * save the extra parsing overhead.
044 */
045@InterfaceAudience.Private
046public abstract class MasterStateStore {
047
048  private static final Logger LOG = LoggerFactory.getLogger(MasterStateStore.class);
049
050  private static final byte[] QUALIFIER = Bytes.toBytes("d");
051
052  private final MasterRegion masterRegion;
053
054  private final byte[] stateName;
055
056  protected MasterStateStore(MasterRegion masterRegion, String stateName, ZKWatcher watcher,
057    String zkPath) throws IOException, KeeperException {
058    this.masterRegion = masterRegion;
059    this.stateName = Bytes.toBytes(stateName);
060    tryMigrate(watcher, zkPath);
061  }
062
063  protected final byte[] getState() throws IOException {
064    return get().getValue(STATE_FAMILY, QUALIFIER);
065  }
066
067  protected final void setState(byte[] state) throws IOException {
068    update(state);
069  }
070
071  private Result get() throws IOException {
072    return masterRegion.get(new Get(stateName).addColumn(STATE_FAMILY, QUALIFIER));
073  }
074
075  private void update(byte[] s) throws IOException {
076    masterRegion.update(r -> r.put(new Put(stateName).addColumn(STATE_FAMILY, QUALIFIER, s)));
077  }
078
079  private byte[] migrate(ZKWatcher watcher, String zkPath) throws KeeperException, IOException {
080    byte[] zkData = ZKUtil.getDataNoWatch(watcher, zkPath, null);
081    if (zkData == null || zkData.length == 0) {
082      return null;
083    }
084    update(zkData);
085    return zkData;
086  }
087
088  private void tryMigrate(ZKWatcher watcher, String zkPath) throws IOException, KeeperException {
089    if (zkPath == null) {
090      // this means we do not store this state in zk, skip migrating
091      return;
092    }
093    Result result = get();
094    if (result.isEmpty()) {
095      // migrate
096      migrate(watcher, zkPath);
097    }
098    // we may fail in the middle so even if the value is available in master local region, we may
099    // still leave a znode on zookeeper, so always try to delete the znode here since it is not very
100    // expensive
101    try {
102      ZKUtil.deleteNodeFailSilent(watcher, zkPath);
103    } catch (Exception e) {
104      LOG.warn("failed to delete migrated zk state node {}, ignore and continue", zkPath);
105    }
106  }
107}