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 java.util.concurrent.locks.Lock;
023import java.util.concurrent.locks.ReentrantLock;
024import org.apache.hadoop.hbase.HConstants;
025import org.apache.hadoop.hbase.ServerName;
026import org.apache.hadoop.hbase.TableName;
027import org.apache.hadoop.hbase.client.DoNotRetryRegionException;
028import org.apache.hadoop.hbase.client.RegionInfo;
029import org.apache.hadoop.hbase.client.RegionOfflineException;
030import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
031import org.apache.hadoop.hbase.master.RegionState;
032import org.apache.hadoop.hbase.master.RegionState.State;
033import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
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  final Lock lock = new ReentrantLock();
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  }
110
111  /**
112   * @param update   new region state this node should be assigned.
113   * @param expected current state should be in this given list of expected states
114   * @return true, if current state is in expected list; otherwise false.
115   */
116  public boolean setState(final State update, final State... expected) {
117    if (!isInState(expected)) {
118      return false;
119    }
120    this.state = update;
121    this.lastUpdate = EnvironmentEdgeManager.currentTime();
122    return true;
123  }
124
125  /**
126   * Put region into OFFLINE mode (set state and clear location).
127   * @return Last recorded server deploy
128   */
129  public ServerName offline() {
130    setState(State.OFFLINE);
131    return setRegionLocation(null);
132  }
133
134  /**
135   * Set new {@link State} but only if currently in <code>expected</code> State (if not, throw
136   * {@link UnexpectedStateException}.
137   */
138  public void transitionState(final State update, final State... expected)
139    throws UnexpectedStateException {
140    if (!setState(update, expected)) {
141      throw new UnexpectedStateException("Expected " + Arrays.toString(expected)
142        + " so could move to " + update + " but current state=" + getState());
143    }
144  }
145
146  /**
147   * Notice that, we will return true if {@code expected} is empty.
148   * <p/>
149   * This is a bit strange but we need this logic, for example, we can change the state to OPENING
150   * from any state, as in SCP we will not change the state to CLOSED before opening the region.
151   */
152  public boolean isInState(State... expected) {
153    if (expected.length == 0) {
154      return true;
155    }
156    return getState().matches(expected);
157  }
158
159  public boolean isStuck() {
160    return isInState(State.FAILED_OPEN) && getProcedure() != null;
161  }
162
163  public boolean isInTransition() {
164    return getProcedure() != null;
165  }
166
167  /**
168   * Return whether the region has been split and not online.
169   * <p/>
170   * In this method we will test both region info and state, and will return true if either of the
171   * test returns true. Please see the comments in
172   * {@link AssignmentManager#markRegionAsSplit(RegionInfo, ServerName, RegionInfo, RegionInfo)} for
173   * more details on why we need to test two conditions.
174   */
175  public boolean isSplit() {
176    return regionInfo.isSplit() || isInState(State.SPLIT);
177  }
178
179  public long getLastUpdate() {
180    TransitRegionStateProcedure proc = this.procedure;
181    return proc != null ? proc.getLastUpdate() : lastUpdate;
182  }
183
184  public void setLastHost(final ServerName serverName) {
185    this.lastHost = serverName;
186  }
187
188  public void setOpenSeqNum(final long seqId) {
189    this.openSeqNum = seqId;
190  }
191
192  public ServerName setRegionLocation(final ServerName serverName) {
193    ServerName lastRegionLocation = this.regionLocation;
194    if (LOG.isTraceEnabled() && serverName == null) {
195      LOG.trace("Tracking when we are set to null " + this, new Throwable("TRACE"));
196    }
197    this.regionLocation = serverName;
198    this.lastUpdate = EnvironmentEdgeManager.currentTime();
199    return lastRegionLocation;
200  }
201
202  public TransitRegionStateProcedure setProcedure(TransitRegionStateProcedure proc) {
203    assert this.procedure == null;
204    this.procedure = proc;
205    ritMap.put(regionInfo, this);
206    return proc;
207  }
208
209  public void unsetProcedure(TransitRegionStateProcedure proc) {
210    assert this.procedure == proc;
211    this.procedure = null;
212    ritMap.remove(regionInfo, this);
213  }
214
215  public TransitRegionStateProcedure getProcedure() {
216    return procedure;
217  }
218
219  public ProcedureEvent<?> getProcedureEvent() {
220    return event;
221  }
222
223  public RegionInfo getRegionInfo() {
224    return regionInfo;
225  }
226
227  public TableName getTable() {
228    return getRegionInfo().getTable();
229  }
230
231  public boolean isSystemTable() {
232    return getTable().isSystemTable();
233  }
234
235  public ServerName getLastHost() {
236    return lastHost;
237  }
238
239  public ServerName getRegionLocation() {
240    return regionLocation;
241  }
242
243  public State getState() {
244    return state;
245  }
246
247  public long getOpenSeqNum() {
248    return openSeqNum;
249  }
250
251  public int getFormatVersion() {
252    // we don't have any format for now
253    // it should probably be in regionInfo.getFormatVersion()
254    return 0;
255  }
256
257  public RegionState toRegionState() {
258    return new RegionState(getRegionInfo(), getState(), getLastUpdate(), getRegionLocation());
259  }
260
261  @Override
262  public int compareTo(final RegionStateNode other) {
263    // NOTE: RegionInfo sort by table first, so we are relying on that.
264    // we have a TestRegionState#testOrderedByTable() that check for that.
265    return RegionInfo.COMPARATOR.compare(getRegionInfo(), other.getRegionInfo());
266  }
267
268  @Override
269  public int hashCode() {
270    return getRegionInfo().hashCode();
271  }
272
273  @Override
274  public boolean equals(final Object other) {
275    if (this == other) {
276      return true;
277    }
278    if (!(other instanceof RegionStateNode)) {
279      return false;
280    }
281    return compareTo((RegionStateNode) other) == 0;
282  }
283
284  @Override
285  public String toString() {
286    return toDescriptiveString();
287  }
288
289  public String toShortString() {
290    // rit= is the current Region-In-Transition State -- see State enum.
291    return String.format("state=%s, location=%s", getState(), getRegionLocation());
292  }
293
294  public String toDescriptiveString() {
295    return String.format("%s, table=%s, region=%s", toShortString(), getTable(),
296      getRegionInfo().getEncodedName());
297  }
298
299  public void checkOnline() throws DoNotRetryRegionException {
300    RegionInfo ri = getRegionInfo();
301    State s = state;
302    if (s != State.OPEN) {
303      throw new DoNotRetryRegionException(ri.getEncodedName() + " is not OPEN; state=" + s);
304    }
305    if (ri.isSplitParent()) {
306      throw new DoNotRetryRegionException(
307        ri.getEncodedName() + " is not online (splitParent=true)");
308    }
309    if (ri.isSplit()) {
310      throw new DoNotRetryRegionException(ri.getEncodedName() + " has split=true");
311    }
312    if (ri.isOffline()) {
313      // RegionOfflineException is not instance of DNRIOE so wrap it.
314      throw new DoNotRetryRegionException(new RegionOfflineException(ri.getEncodedName()));
315    }
316  }
317
318  public void lock() {
319    lock.lock();
320  }
321
322  public void unlock() {
323    lock.unlock();
324  }
325}