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.assignment;
019
020import java.util.Arrays;
021import java.util.concurrent.ConcurrentMap;
022import org.apache.hadoop.hbase.HConstants;
023import org.apache.hadoop.hbase.ServerName;
024import org.apache.hadoop.hbase.TableName;
025import org.apache.hadoop.hbase.client.DoNotRetryRegionException;
026import org.apache.hadoop.hbase.client.RegionInfo;
027import org.apache.hadoop.hbase.client.RegionOfflineException;
028import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
029import org.apache.hadoop.hbase.master.RegionState;
030import org.apache.hadoop.hbase.master.RegionState.State;
031import org.apache.hadoop.hbase.procedure2.Procedure;
032import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
033import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
034import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
035import org.apache.yetus.audience.InterfaceAudience;
036import org.slf4j.Logger;
037import org.slf4j.LoggerFactory;
038
039/**
040 * Current Region State. Most fields are synchronized with meta region, i.e, we will update meta
041 * immediately after we modify this RegionStateNode, and usually under the lock. The only exception
042 * is {@link #lastHost}, which should not be used for critical condition.
043 * <p/>
044 * Typically, the only way to modify this class is through {@link TransitRegionStateProcedure}, and
045 * we will record the TRSP along with this RegionStateNode to make sure that there could at most one
046 * TRSP. For other operations, such as SCP, we will first get the lock, and then try to schedule a
047 * TRSP. If there is already one, then the solution will be different:
048 * <ul>
049 * <li>For SCP, we will update the region state in meta to tell the TRSP to retry.</li>
050 * <li>For DisableTableProcedure, as we have the xlock, we can make sure that the TRSP has not been
051 * executed yet, so just unset it and attach a new one. The original one will quit immediately when
052 * executing.</li>
053 * <li>For split/merge, we will fail immediately as there is no actual operations yet so no
054 * harm.</li>
055 * <li>For EnableTableProcedure/TruncateTableProcedure, we can make sure that there will be no TRSP
056 * attached with the RSNs.</li>
057 * <li>For other procedures, you'd better use ReopenTableRegionsProcedure. The RTRP will take care
058 * of lots of corner cases when reopening regions.</li>
059 * </ul>
060 * <p/>
061 * Several fields are declared with {@code volatile}, which means you are free to get it without
062 * lock, but usually you should not use these fields without locking for critical condition, as it
063 * will be easily to introduce inconsistency. For example, you are free to dump the status and show
064 * it on web without locking, but if you want to change the state of the RegionStateNode by checking
065 * the current state, you'd better have the lock...
066 */
067@InterfaceAudience.Private
068public class RegionStateNode implements Comparable<RegionStateNode> {
069
070  private static final Logger LOG = LoggerFactory.getLogger(RegionStateNode.class);
071
072  private static final class AssignmentProcedureEvent extends ProcedureEvent<RegionInfo> {
073    public AssignmentProcedureEvent(final RegionInfo regionInfo) {
074      super(regionInfo);
075    }
076  }
077
078  private final RegionStateNodeLock lock;
079  private final RegionInfo regionInfo;
080  private final ProcedureEvent<?> event;
081  private final ConcurrentMap<RegionInfo, RegionStateNode> ritMap;
082
083  // volatile only for getLastUpdate and test usage, the upper layer should sync on the
084  // RegionStateNode before accessing usually.
085  private volatile TransitRegionStateProcedure procedure = null;
086  private volatile ServerName regionLocation = null;
087  // notice that, the lastHost will only be updated when a region is successfully CLOSED through
088  // UnassignProcedure, so do not use it for critical condition as the data maybe stale and unsync
089  // with the data in meta.
090  private volatile ServerName lastHost = null;
091  /**
092   * A Region-in-Transition (RIT) moves through states. See {@link State} for complete list. A
093   * Region that is opened moves from OFFLINE => OPENING => OPENED.
094   */
095  private volatile State state = State.OFFLINE;
096
097  /**
098   * Updated whenever a call to {@link #setRegionLocation(ServerName)} or
099   * {@link #setState(RegionState.State, RegionState.State...)}.
100   */
101  private volatile long lastUpdate = 0;
102
103  private volatile long openSeqNum = HConstants.NO_SEQNUM;
104
105  RegionStateNode(RegionInfo regionInfo, ConcurrentMap<RegionInfo, RegionStateNode> ritMap) {
106    this.regionInfo = regionInfo;
107    this.event = new AssignmentProcedureEvent(regionInfo);
108    this.ritMap = ritMap;
109    this.lock = new RegionStateNodeLock(regionInfo);
110  }
111
112  /**
113   * @param update   new region state this node should be assigned.
114   * @param expected current state should be in this given list of expected states
115   * @return true, if current state is in expected list; otherwise false.
116   */
117  public boolean setState(final State update, final State... expected) {
118    if (!isInState(expected)) {
119      return false;
120    }
121    this.state = update;
122    this.lastUpdate = EnvironmentEdgeManager.currentTime();
123    return true;
124  }
125
126  /**
127   * Put region into OFFLINE mode (set state and clear location).
128   * @return Last recorded server deploy
129   */
130  public ServerName offline() {
131    setState(State.OFFLINE);
132    return setRegionLocation(null);
133  }
134
135  /**
136   * Set new {@link State} but only if currently in <code>expected</code> State (if not, throw
137   * {@link UnexpectedStateException}.
138   */
139  public void transitionState(final State update, final State... expected)
140    throws UnexpectedStateException {
141    if (!setState(update, expected)) {
142      throw new UnexpectedStateException("Expected " + Arrays.toString(expected)
143        + " so could move to " + update + " but current state=" + getState());
144    }
145  }
146
147  /**
148   * Notice that, we will return true if {@code expected} is empty.
149   * <p/>
150   * This is a bit strange but we need this logic, for example, we can change the state to OPENING
151   * from any state, as in SCP we will not change the state to CLOSED before opening the region.
152   */
153  public boolean isInState(State... expected) {
154    if (expected.length == 0) {
155      return true;
156    }
157    return getState().matches(expected);
158  }
159
160  public boolean isStuck() {
161    return isInState(State.FAILED_OPEN) && getProcedure() != null;
162  }
163
164  public boolean isInTransition() {
165    return getProcedure() != null;
166  }
167
168  /**
169   * Return whether the region has been split and not online.
170   * <p/>
171   * In this method we will test both region info and state, and will return true if either of the
172   * test returns true. Please see the comments in
173   * {@link AssignmentManager#markRegionAsSplit(RegionInfo, ServerName, RegionInfo, RegionInfo)} for
174   * more details on why we need to test two conditions.
175   */
176  public boolean isSplit() {
177    return regionInfo.isSplit() || isInState(State.SPLIT);
178  }
179
180  public long getLastUpdate() {
181    TransitRegionStateProcedure proc = this.procedure;
182    if (proc != null) {
183      long lastUpdate = proc.getLastUpdate();
184      return lastUpdate != 0 ? lastUpdate : proc.getSubmittedTime();
185    }
186    return lastUpdate;
187  }
188
189  public void setLastHost(final ServerName serverName) {
190    this.lastHost = serverName;
191  }
192
193  public void setOpenSeqNum(final long seqId) {
194    this.openSeqNum = seqId;
195  }
196
197  public ServerName setRegionLocation(final ServerName serverName) {
198    ServerName lastRegionLocation = this.regionLocation;
199    if (LOG.isTraceEnabled() && serverName == null) {
200      LOG.trace("Tracking when we are set to null " + this, new Throwable("TRACE"));
201    }
202    this.regionLocation = serverName;
203    this.lastUpdate = EnvironmentEdgeManager.currentTime();
204    return lastRegionLocation;
205  }
206
207  public TransitRegionStateProcedure setProcedure(TransitRegionStateProcedure proc) {
208    assert this.procedure == null;
209    this.procedure = proc;
210    ritMap.put(regionInfo, this);
211    return proc;
212  }
213
214  public void unsetProcedure(TransitRegionStateProcedure proc) {
215    assert this.procedure == proc;
216    this.procedure = null;
217    ritMap.remove(regionInfo, this);
218  }
219
220  public TransitRegionStateProcedure getProcedure() {
221    return procedure;
222  }
223
224  public ProcedureEvent<?> getProcedureEvent() {
225    return event;
226  }
227
228  public RegionInfo getRegionInfo() {
229    return regionInfo;
230  }
231
232  public TableName getTable() {
233    return getRegionInfo().getTable();
234  }
235
236  public boolean isSystemTable() {
237    return getTable().isSystemTable();
238  }
239
240  public ServerName getLastHost() {
241    return lastHost;
242  }
243
244  public ServerName getRegionLocation() {
245    return regionLocation;
246  }
247
248  public String getRegionServerName() {
249    ServerName sn = getRegionLocation();
250    if (sn != null) {
251      return sn.getServerName();
252    }
253    return null;
254  }
255
256  public State getState() {
257    return state;
258  }
259
260  public long getOpenSeqNum() {
261    return openSeqNum;
262  }
263
264  public int getFormatVersion() {
265    // we don't have any format for now
266    // it should probably be in regionInfo.getFormatVersion()
267    return 0;
268  }
269
270  public RegionState toRegionState() {
271    return new RegionState(getRegionInfo(), getState(), getLastUpdate(), getRegionLocation());
272  }
273
274  @Override
275  public int compareTo(final RegionStateNode other) {
276    // NOTE: RegionInfo sort by table first, so we are relying on that.
277    // we have a TestRegionState#testOrderedByTable() that check for that.
278    return RegionInfo.COMPARATOR.compare(getRegionInfo(), other.getRegionInfo());
279  }
280
281  @Override
282  public int hashCode() {
283    return getRegionInfo().hashCode();
284  }
285
286  @Override
287  public boolean equals(final Object other) {
288    if (this == other) {
289      return true;
290    }
291    if (!(other instanceof RegionStateNode)) {
292      return false;
293    }
294    return compareTo((RegionStateNode) other) == 0;
295  }
296
297  @Override
298  public String toString() {
299    return toDescriptiveString();
300  }
301
302  public String toShortString() {
303    // rit= is the current Region-In-Transition State -- see State enum.
304    return String.format("state=%s, location=%s", getState(), getRegionLocation());
305  }
306
307  public String toDescriptiveString() {
308    return String.format("%s, table=%s, region=%s", toShortString(), getTable(),
309      getRegionInfo().getEncodedName());
310  }
311
312  public void checkOnline() throws DoNotRetryRegionException {
313    RegionInfo ri = getRegionInfo();
314    State s = state;
315    if (s != State.OPEN) {
316      throw new DoNotRetryRegionException(ri.getEncodedName() + " is not OPEN; state=" + s);
317    }
318    if (ri.isSplitParent()) {
319      throw new DoNotRetryRegionException(
320        ri.getEncodedName() + " is not online (splitParent=true)");
321    }
322    if (ri.isSplit()) {
323      throw new DoNotRetryRegionException(ri.getEncodedName() + " has split=true");
324    }
325    if (ri.isOffline()) {
326      // RegionOfflineException is not instance of DNRIOE so wrap it.
327      throw new DoNotRetryRegionException(new RegionOfflineException(ri.getEncodedName()));
328    }
329  }
330
331  // The below 3 methods are for normal locking operation, where the thread owner is the current
332  // thread. Typically you just need to use these 3 methods, and use try..finally to release the
333  // lock in the finally block
334  /**
335   * @see RegionStateNodeLock#lock()
336   */
337  public void lock() {
338    lock.lock();
339  }
340
341  /**
342   * @see RegionStateNodeLock#tryLock()
343   */
344  public boolean tryLock() {
345    return lock.tryLock();
346  }
347
348  /**
349   * @see RegionStateNodeLock#unlock()
350   */
351  public void unlock() {
352    lock.unlock();
353  }
354
355  // The below 3 methods are for locking region state node when executing procedures, where we may
356  // do some time consuming work under the lock, for example, updating meta. As we may suspend the
357  // procedure while holding the lock and then release it when the procedure is back, in another
358  // thread, so we need to use the procedure itself as owner, instead of the current thread. You can
359  // see the usage in TRSP, SCP, and RegionRemoteProcedureBase for more details.
360  // Notice that, this does not mean you must use these 3 methods when locking region state node in
361  // procedure, you are free to use the above 3 methods if you do not want to hold the lock when
362  // suspending the procedure.
363  /**
364   * @see RegionStateNodeLock#lock(Procedure, Runnable)
365   */
366  public void lock(Procedure<?> proc, Runnable wakeUp) throws ProcedureSuspendedException {
367    lock.lock(proc, wakeUp);
368  }
369
370  /**
371   * @see RegionStateNodeLock#tryLock(Procedure)
372   */
373  public boolean tryLock(Procedure<?> proc) {
374    return lock.tryLock(proc);
375  }
376
377  /**
378   * @see RegionStateNodeLock#unlock(Procedure)
379   */
380  public void unlock(Procedure<?> proc) {
381    lock.unlock(proc);
382  }
383
384  /**
385   * @see RegionStateNodeLock#isLocked()
386   */
387  boolean isLocked() {
388    return lock.isLocked();
389  }
390
391  /**
392   * @see RegionStateNodeLock#isLockedBy(Object)
393   */
394  public boolean isLockedBy(Procedure<?> proc) {
395    return lock.isLockedBy(proc);
396  }
397}