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 edu.umd.cs.findbugs.annotations.NonNull;
021import edu.umd.cs.findbugs.annotations.Nullable;
022import java.io.IOException;
023import java.util.HashMap;
024import java.util.Map;
025import java.util.Set;
026import java.util.concurrent.ConcurrentHashMap;
027import java.util.concurrent.ConcurrentMap;
028import java.util.concurrent.locks.ReadWriteLock;
029import org.apache.hadoop.hbase.MetaTableAccessor;
030import org.apache.hadoop.hbase.TableDescriptors;
031import org.apache.hadoop.hbase.TableName;
032import org.apache.hadoop.hbase.TableNotFoundException;
033import org.apache.hadoop.hbase.client.Connection;
034import org.apache.hadoop.hbase.client.Result;
035import org.apache.hadoop.hbase.client.TableDescriptor;
036import org.apache.hadoop.hbase.client.TableState;
037import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
038import org.apache.hadoop.hbase.util.IdReadWriteLock;
039import org.apache.hadoop.hbase.util.ZKDataMigrator;
040import org.apache.hadoop.hbase.zookeeper.ZKUtil;
041import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
042import org.apache.yetus.audience.InterfaceAudience;
043import org.apache.zookeeper.KeeperException;
044import org.slf4j.Logger;
045import org.slf4j.LoggerFactory;
046
047import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
048
049/**
050 * This is a helper class used to manage table states. This class uses hbase:meta as its store for
051 * table state so hbase:meta must be online before {@link #start()} is called.
052 */
053// TODO: Make this a guava Service
054@InterfaceAudience.Private
055public class TableStateManager {
056
057  private static final Logger LOG = LoggerFactory.getLogger(TableStateManager.class);
058  /**
059   * Set this key to false in Configuration to disable migrating table state from zookeeper so
060   * hbase:meta table.
061   */
062  private static final String MIGRATE_TABLE_STATE_FROM_ZK_KEY =
063    "hbase.migrate.table.state.from.zookeeper";
064
065  private final IdReadWriteLock<TableName> tnLock = new IdReadWriteLock<>();
066  protected final MasterServices master;
067
068  private final ConcurrentMap<TableName, TableState.State> tableName2State =
069    new ConcurrentHashMap<>();
070
071  public TableStateManager(MasterServices master) {
072    this.master = master;
073  }
074
075  /**
076   * Set table state to provided. Caller should lock table on write.
077   * @param tableName table to change state for
078   * @param newState new state
079   */
080  public void setTableState(TableName tableName, TableState.State newState) throws IOException {
081    ReadWriteLock lock = tnLock.getLock(tableName);
082    lock.writeLock().lock();
083    try {
084      updateMetaState(tableName, newState);
085    } finally {
086      lock.writeLock().unlock();
087    }
088  }
089
090  /**
091   * Set table state to provided but only if table in specified states Caller should lock table on
092   * write.
093   * @param tableName table to change state for
094   * @param newState new state
095   * @param states states to check against
096   * @return null if succeed or table state if failed
097   */
098  public TableState setTableStateIfInStates(TableName tableName, TableState.State newState,
099      TableState.State... states) throws IOException {
100    ReadWriteLock lock = tnLock.getLock(tableName);
101    lock.writeLock().lock();
102    try {
103      TableState currentState = readMetaState(tableName);
104      if (currentState == null) {
105        throw new TableNotFoundException(tableName);
106      }
107      if (currentState.inStates(states)) {
108        updateMetaState(tableName, newState);
109        return null;
110      } else {
111        return currentState;
112      }
113    } finally {
114      lock.writeLock().unlock();
115    }
116  }
117
118  /**
119   * Set table state to provided but only if table not in specified states Caller should lock table
120   * on write.
121   * @param tableName table to change state for
122   * @param newState new state
123   * @param states states to check against
124   */
125  public boolean setTableStateIfNotInStates(TableName tableName, TableState.State newState,
126      TableState.State... states) throws IOException {
127    ReadWriteLock lock = tnLock.getLock(tableName);
128    lock.writeLock().lock();
129    try {
130      TableState currentState = readMetaState(tableName);
131      if (currentState == null) {
132        throw new TableNotFoundException(tableName);
133      }
134      if (!currentState.inStates(states)) {
135        updateMetaState(tableName, newState);
136        return true;
137      } else {
138        return false;
139      }
140    } finally {
141      lock.writeLock().unlock();
142    }
143  }
144
145  public boolean isTableState(TableName tableName, TableState.State... states) {
146    try {
147      TableState tableState = getTableState(tableName);
148      return tableState.isInStates(states);
149    } catch (IOException e) {
150      LOG.error("Unable to get table " + tableName + " state", e);
151      // XXX: is it safe to just return false here?
152      return false;
153    }
154  }
155
156  public void setDeletedTable(TableName tableName) throws IOException {
157    if (tableName.equals(TableName.META_TABLE_NAME)) {
158      return;
159    }
160    ReadWriteLock lock = tnLock.getLock(tableName);
161    lock.writeLock().lock();
162    try {
163      MetaTableAccessor.deleteTableState(master.getConnection(), tableName);
164      metaStateDeleted(tableName);
165    } finally {
166      tableName2State.remove(tableName);
167      lock.writeLock().unlock();
168    }
169  }
170
171  public boolean isTablePresent(TableName tableName) throws IOException {
172    ReadWriteLock lock = tnLock.getLock(tableName);
173    lock.readLock().lock();
174    try {
175      return readMetaState(tableName) != null;
176    } finally {
177      lock.readLock().unlock();
178    }
179  }
180
181  /**
182   * Return all tables in given states.
183   * @param states filter by states
184   * @return tables in given states
185   */
186  public Set<TableName> getTablesInStates(TableState.State... states) throws IOException {
187    // Only be called in region normalizer, will not use cache.
188    final Set<TableName> rv = Sets.newHashSet();
189    MetaTableAccessor.fullScanTables(master.getConnection(), new MetaTableAccessor.Visitor() {
190      @Override
191      public boolean visit(Result r) throws IOException {
192        TableState tableState = MetaTableAccessor.getTableState(r);
193        if (tableState != null && tableState.inStates(states)) {
194          rv.add(tableState.getTableName());
195        }
196        return true;
197      }
198    });
199    return rv;
200  }
201
202  public static class TableStateNotFoundException extends TableNotFoundException {
203    TableStateNotFoundException(TableName tableName) {
204      super(tableName.getNameAsString());
205    }
206  }
207
208  @NonNull
209  public TableState getTableState(TableName tableName) throws IOException {
210    ReadWriteLock lock = tnLock.getLock(tableName);
211    lock.readLock().lock();
212    try {
213      TableState currentState = readMetaState(tableName);
214      if (currentState == null) {
215        throw new TableStateNotFoundException(tableName);
216      }
217      return currentState;
218    } finally {
219      lock.readLock().unlock();
220    }
221  }
222
223  private void updateMetaState(TableName tableName, TableState.State newState) throws IOException {
224    if (tableName.equals(TableName.META_TABLE_NAME)) {
225      if (TableState.State.DISABLING.equals(newState) ||
226        TableState.State.DISABLED.equals(newState)) {
227        throw new IllegalArgumentIOException("Cannot disable the meta table; " + newState);
228      }
229      // Otherwise, just return; no need to set ENABLED on meta -- it is always ENABLED.
230      return;
231    }
232    boolean succ = false;
233    try {
234      MetaTableAccessor.updateTableState(master.getConnection(), tableName, newState);
235      tableName2State.put(tableName, newState);
236      succ = true;
237    } finally {
238      if (!succ) {
239        tableName2State.remove(tableName);
240      }
241    }
242    metaStateUpdated(tableName, newState);
243  }
244
245  protected void metaStateUpdated(TableName tableName, TableState.State newState)
246      throws IOException {
247  }
248
249  protected void metaStateDeleted(TableName tableName) throws IOException {
250  }
251
252  @Nullable
253  private TableState readMetaState(TableName tableName) throws IOException {
254    TableState.State state = tableName2State.get(tableName);
255    if (state != null) {
256      return new TableState(tableName, state);
257    }
258    TableState tableState = MetaTableAccessor.getTableState(master.getConnection(), tableName);
259    if (tableState != null) {
260      tableName2State.putIfAbsent(tableName, tableState.getState());
261    }
262    return tableState;
263  }
264
265  public void start() throws IOException {
266    TableDescriptors tableDescriptors = master.getTableDescriptors();
267    migrateZooKeeper();
268    Connection connection = master.getConnection();
269    fixTableStates(tableDescriptors, connection);
270  }
271
272  private void fixTableStates(TableDescriptors tableDescriptors, Connection connection)
273      throws IOException {
274    Map<String, TableState> states = new HashMap<>();
275    // NOTE: Full hbase:meta table scan!
276    MetaTableAccessor.fullScanTables(connection, new MetaTableAccessor.Visitor() {
277      @Override
278      public boolean visit(Result r) throws IOException {
279        TableState state = MetaTableAccessor.getTableState(r);
280        states.put(state.getTableName().getNameAsString(), state);
281        return true;
282      }
283    });
284    for (TableDescriptor tableDesc : tableDescriptors.getAll().values()) {
285      TableName tableName = tableDesc.getTableName();
286      if (TableName.isMetaTableName(tableName)) {
287        // This table is always enabled. No fixup needed. No entry in hbase:meta needed.
288        // Call through to fixTableState though in case a super class wants to do something.
289        fixTableState(new TableState(tableName, TableState.State.ENABLED));
290        continue;
291      }
292      TableState tableState = states.get(tableName.getNameAsString());
293      if (tableState == null) {
294        LOG.warn(tableName + " has no table state in hbase:meta, assuming ENABLED");
295        MetaTableAccessor.updateTableState(connection, tableName, TableState.State.ENABLED);
296        fixTableState(new TableState(tableName, TableState.State.ENABLED));
297        tableName2State.put(tableName, TableState.State.ENABLED);
298      } else {
299        fixTableState(tableState);
300        tableName2State.put(tableName, tableState.getState());
301      }
302    }
303  }
304
305  /**
306   * For subclasses in case they want to do fixup post hbase:meta.
307   */
308  protected void fixTableState(TableState tableState) throws IOException {
309  }
310
311  /**
312   * This code is for case where a hbase2 Master is starting for the first time. ZooKeeper is where
313   * we used to keep table state. On first startup, read zookeeper and update hbase:meta with the
314   * table states found in zookeeper. This is tricky as we'll do this check every time we startup
315   * until mirroring is disabled. See the {@link #MIGRATE_TABLE_STATE_FROM_ZK_KEY} flag. Original
316   * form of this migration came in with HBASE-13032. It deleted all znodes when done. We can't do
317   * that if we want to support hbase-1.x clients who need to be able to read table state out of zk.
318   * See {@link MirroringTableStateManager}.
319   * @deprecated Since 2.0.0. Remove in hbase-3.0.0.
320   */
321  @Deprecated
322  private void migrateZooKeeper() throws IOException {
323    if (!this.master.getConfiguration().getBoolean(MIGRATE_TABLE_STATE_FROM_ZK_KEY, true)) {
324      return;
325    }
326    try {
327      for (Map.Entry<TableName, TableState.State> entry : ZKDataMigrator
328        .queryForTableStates(this.master.getZooKeeper()).entrySet()) {
329        if (this.master.getTableDescriptors().get(entry.getKey()) == null) {
330          deleteZooKeeper(entry.getKey());
331          LOG.info("Purged table state entry from zookeepr for table not in hbase:meta: " +
332            entry.getKey());
333          continue;
334        }
335        TableState ts = null;
336        try {
337          ts = getTableState(entry.getKey());
338        } catch (TableStateNotFoundException e) {
339          // This can happen; table exists but no TableState.
340        }
341        if (ts == null) {
342          TableState.State zkstate = entry.getValue();
343          // Only migrate if it is an enable or disabled table. If in-between -- ENABLING or
344          // DISABLING then we have a problem; we are starting up an hbase-2 on a cluster with
345          // RIT. It is going to be rough!
346          if (zkstate.equals(TableState.State.ENABLED) ||
347            zkstate.equals(TableState.State.DISABLED)) {
348            LOG.info("Migrating table state from zookeeper to hbase:meta; tableName=" +
349              entry.getKey() + ", state=" + entry.getValue());
350            updateMetaState(entry.getKey(), entry.getValue());
351          } else {
352            LOG.warn("Table={} has no state and zookeeper state is in-between={} (neither " +
353              "ENABLED or DISABLED); NOT MIGRATING table state", entry.getKey(), zkstate);
354          }
355        }
356        // What if the table states disagree? Defer to the hbase:meta setting rather than have the
357        // hbase-1.x support prevail.
358      }
359    } catch (KeeperException | InterruptedException e) {
360      LOG.warn("Failed reading table state from zookeeper", e);
361    }
362  }
363
364  /**
365   * Utility method that knows how to delete the old hbase-1.x table state znode. Used also by the
366   * Mirroring subclass.
367   * @deprecated Since 2.0.0. To be removed in hbase-3.0.0.
368   */
369  @Deprecated
370  protected void deleteZooKeeper(TableName tableName) {
371    try {
372      // Delete from ZooKeeper
373      String znode = ZNodePaths.joinZNode(this.master.getZooKeeper().getZNodePaths().tableZNode,
374        tableName.getNameAsString());
375      ZKUtil.deleteNodeFailSilent(this.master.getZooKeeper(), znode);
376    } catch (KeeperException e) {
377      LOG.warn("Failed deleting table state from zookeeper", e);
378    }
379  }
380}