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  public long getLastUpdate() {
168    TransitRegionStateProcedure proc = this.procedure;
169    return proc != null ? proc.getLastUpdate() : lastUpdate;
170  }
171
172  public void setLastHost(final ServerName serverName) {
173    this.lastHost = serverName;
174  }
175
176  public void setOpenSeqNum(final long seqId) {
177    this.openSeqNum = seqId;
178  }
179
180  public ServerName setRegionLocation(final ServerName serverName) {
181    ServerName lastRegionLocation = this.regionLocation;
182    if (LOG.isTraceEnabled() && serverName == null) {
183      LOG.trace("Tracking when we are set to null " + this, new Throwable("TRACE"));
184    }
185    this.regionLocation = serverName;
186    this.lastUpdate = EnvironmentEdgeManager.currentTime();
187    return lastRegionLocation;
188  }
189
190  public TransitRegionStateProcedure setProcedure(TransitRegionStateProcedure proc) {
191    assert this.procedure == null;
192    this.procedure = proc;
193    ritMap.put(regionInfo, this);
194    return proc;
195  }
196
197  public void unsetProcedure(TransitRegionStateProcedure proc) {
198    assert this.procedure == proc;
199    this.procedure = null;
200    ritMap.remove(regionInfo, this);
201  }
202
203  public TransitRegionStateProcedure getProcedure() {
204    return procedure;
205  }
206
207  public ProcedureEvent<?> getProcedureEvent() {
208    return event;
209  }
210
211  public RegionInfo getRegionInfo() {
212    return regionInfo;
213  }
214
215  public TableName getTable() {
216    return getRegionInfo().getTable();
217  }
218
219  public boolean isSystemTable() {
220    return getTable().isSystemTable();
221  }
222
223  public ServerName getLastHost() {
224    return lastHost;
225  }
226
227  public ServerName getRegionLocation() {
228    return regionLocation;
229  }
230
231  public State getState() {
232    return state;
233  }
234
235  public long getOpenSeqNum() {
236    return openSeqNum;
237  }
238
239  public int getFormatVersion() {
240    // we don't have any format for now
241    // it should probably be in regionInfo.getFormatVersion()
242    return 0;
243  }
244
245  public RegionState toRegionState() {
246    return new RegionState(getRegionInfo(), getState(), getLastUpdate(), getRegionLocation());
247  }
248
249  @Override
250  public int compareTo(final RegionStateNode other) {
251    // NOTE: RegionInfo sort by table first, so we are relying on that.
252    // we have a TestRegionState#testOrderedByTable() that check for that.
253    return RegionInfo.COMPARATOR.compare(getRegionInfo(), other.getRegionInfo());
254  }
255
256  @Override
257  public int hashCode() {
258    return getRegionInfo().hashCode();
259  }
260
261  @Override
262  public boolean equals(final Object other) {
263    if (this == other) {
264      return true;
265    }
266    if (!(other instanceof RegionStateNode)) {
267      return false;
268    }
269    return compareTo((RegionStateNode) other) == 0;
270  }
271
272  @Override
273  public String toString() {
274    return toDescriptiveString();
275  }
276
277  public String toShortString() {
278    // rit= is the current Region-In-Transition State -- see State enum.
279    return String.format("state=%s, location=%s", getState(), getRegionLocation());
280  }
281
282  public String toDescriptiveString() {
283    return String.format("%s, table=%s, region=%s", toShortString(), getTable(),
284      getRegionInfo().getEncodedName());
285  }
286
287  public void checkOnline() throws DoNotRetryRegionException {
288    RegionInfo ri = getRegionInfo();
289    State s = state;
290    if (s != State.OPEN) {
291      throw new DoNotRetryRegionException(ri.getEncodedName() + " is not OPEN; state=" + s);
292    }
293    if (ri.isSplitParent()) {
294      throw new DoNotRetryRegionException(
295        ri.getEncodedName() + " is not online (splitParent=true)");
296    }
297    if (ri.isSplit()) {
298      throw new DoNotRetryRegionException(ri.getEncodedName() + " has split=true");
299    }
300    if (ri.isOffline()) {
301      // RegionOfflineException is not instance of DNRIOE so wrap it.
302      throw new DoNotRetryRegionException(new RegionOfflineException(ri.getEncodedName()));
303    }
304  }
305
306  public void lock() {
307    lock.lock();
308  }
309
310  public void unlock() {
311    lock.unlock();
312  }
313}