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.janitor;
019
020import java.io.IOException;
021import org.apache.hadoop.hbase.HConstants;
022import org.apache.hadoop.hbase.HRegionLocation;
023import org.apache.hadoop.hbase.MetaTableAccessor;
024import org.apache.hadoop.hbase.RegionLocations;
025import org.apache.hadoop.hbase.ServerName;
026import org.apache.hadoop.hbase.client.RegionInfo;
027import org.apache.hadoop.hbase.client.RegionInfoBuilder;
028import org.apache.hadoop.hbase.client.Result;
029import org.apache.hadoop.hbase.client.TableState;
030import org.apache.hadoop.hbase.master.MasterServices;
031import org.apache.hadoop.hbase.master.RegionState;
032import org.apache.hadoop.hbase.master.ServerManager;
033import org.apache.hadoop.hbase.util.Bytes;
034import org.apache.hadoop.hbase.util.Pair;
035import org.apache.yetus.audience.InterfaceAudience;
036import org.slf4j.Logger;
037import org.slf4j.LoggerFactory;
038
039/**
040 * Visitor we use in here in CatalogJanitor to go against hbase:meta table. Generates a Report made
041 * of a collection of split parents and counts of rows in the hbase:meta table. Also runs hbase:meta
042 * consistency checks to generate more report. Report is NOT ready until after this visitor has been
043 * {@link #close()}'d.
044 */
045@InterfaceAudience.Private
046class ReportMakingVisitor implements MetaTableAccessor.CloseableVisitor {
047
048  private static final Logger LOG = LoggerFactory.getLogger(ReportMakingVisitor.class);
049
050  private final MasterServices services;
051  private volatile boolean closed;
052
053  /**
054   * Report is not done until after the close has been called.
055   */
056  private CatalogJanitorReport report = new CatalogJanitorReport();
057
058  /**
059   * RegionInfo from previous row.
060   */
061  private RegionInfo previous = null;
062
063  /**
064   * Keep account of the highest end key seen as we move through hbase:meta. Usually, the current
065   * RegionInfo has the highest end key but if an overlap, this may no longer hold. An overlap may
066   * be a region with startkey 'd' and endkey 'g'. The next region in meta may be 'e' to 'f' and
067   * then 'f' to 'g'. Looking at previous and current meta row, we won't know about the 'd' to 'g'
068   * overlap unless we keep a running 'highest-endpoint-seen'.
069   */
070  private RegionInfo highestEndKeyRegionInfo = null;
071
072  ReportMakingVisitor(MasterServices services) {
073    this.services = services;
074  }
075
076  /**
077   * Do not call until after {@link #close()}. Will throw a {@link RuntimeException} if you do.
078   */
079  CatalogJanitorReport getReport() {
080    if (!this.closed) {
081      throw new RuntimeException("Report not ready until after close()");
082    }
083    return this.report;
084  }
085
086  @Override
087  public boolean visit(Result r) {
088    if (r == null || r.isEmpty()) {
089      return true;
090    }
091    this.report.count++;
092    RegionInfo regionInfo = null;
093    try {
094      regionInfo = metaTableConsistencyCheck(r);
095    } catch (Throwable t) {
096      LOG.warn("Failed consistency check on {}", Bytes.toStringBinary(r.getRow()), t);
097    }
098    if (regionInfo != null) {
099      LOG.trace(regionInfo.toString());
100      if (regionInfo.isSplitParent()) { // splitParent means split and offline.
101        this.report.splitParents.put(regionInfo, r);
102      }
103      if (MetaTableAccessor.hasMergeRegions(r.rawCells())) {
104        this.report.mergedRegions.put(regionInfo, r);
105      }
106    }
107    // Returning true means "keep scanning"
108    return true;
109  }
110
111  /**
112   * Check row.
113   * @param metaTableRow Row from hbase:meta table.
114   * @return Returns default regioninfo found in row parse as a convenience to save on having to do
115   *         a double-parse of Result.
116   */
117  private RegionInfo metaTableConsistencyCheck(Result metaTableRow) {
118    RegionInfo ri;
119    // Locations comes back null if the RegionInfo field is empty.
120    // If locations is null, ensure the regioninfo is for sure empty before progressing.
121    // If really empty, report as missing regioninfo! Otherwise, can run server check
122    // and get RegionInfo from locations.
123    RegionLocations locations = MetaTableAccessor.getRegionLocations(metaTableRow);
124    if (locations == null) {
125      ri = MetaTableAccessor.getRegionInfo(metaTableRow, HConstants.REGIONINFO_QUALIFIER);
126    } else {
127      ri = locations.getDefaultRegionLocation().getRegion();
128      checkServer(locations);
129    }
130
131    if (ri == null) {
132      this.report.emptyRegionInfo.add(metaTableRow.getRow());
133      return ri;
134    }
135
136    if (!Bytes.equals(metaTableRow.getRow(), ri.getRegionName())) {
137      LOG.warn(
138        "INCONSISTENCY: Row name is not equal to serialized info:regioninfo content; "
139          + "row={} {}; See if RegionInfo is referenced in another hbase:meta row? Delete?",
140        Bytes.toStringBinary(metaTableRow.getRow()), ri.getRegionNameAsString());
141      return null;
142    }
143    // Skip split parent region
144    if (ri.isSplitParent()) {
145      return ri;
146    }
147    // If table is disabled, skip integrity check.
148    if (!isTableDisabled(ri)) {
149      if (isTableTransition(ri)) {
150        // HBCK1 used to have a special category for missing start or end keys.
151        // We'll just lump them in as 'holes'.
152
153        // This is a table transition. If this region is not first region, report a hole.
154        if (!ri.isFirst()) {
155          addHole(RegionInfoBuilder.UNDEFINED, ri);
156        }
157        // This is a table transition. If last region was not last region of previous table,
158        // report a hole
159        if (this.previous != null && !this.previous.isLast()) {
160          addHole(this.previous, RegionInfoBuilder.UNDEFINED);
161        }
162      } else {
163        if (!this.previous.isNext(ri)) {
164          if (this.previous.isOverlap(ri)) {
165            addOverlap(this.previous, ri);
166          } else if (ri.isOverlap(this.highestEndKeyRegionInfo)) {
167            // We may have seen a region a few rows back that overlaps this one.
168            addOverlap(this.highestEndKeyRegionInfo, ri);
169          } else if (!this.highestEndKeyRegionInfo.isNext(ri)) {
170            // Need to check the case if this.highestEndKeyRegionInfo.isNext(ri). If no,
171            // report a hole, otherwise, it is ok. For an example,
172            // previous: [aa, bb), ri: [cc, dd), highestEndKeyRegionInfo: [a, cc)
173            // In this case, it should not report a hole, as highestEndKeyRegionInfo covers
174            // the hole between previous and ri.
175            addHole(this.previous, ri);
176          }
177        } else if (ri.isOverlap(this.highestEndKeyRegionInfo)) {
178          // We may have seen a region a few rows back that overlaps this one
179          // even though it properly 'follows' the region just before.
180          addOverlap(this.highestEndKeyRegionInfo, ri);
181        }
182      }
183    }
184    this.previous = ri;
185    this.highestEndKeyRegionInfo =
186      MetaFixer.getRegionInfoWithLargestEndKey(this.highestEndKeyRegionInfo, ri);
187    return ri;
188  }
189
190  private void addOverlap(RegionInfo a, RegionInfo b) {
191    this.report.overlaps.add(new Pair<>(a, b));
192  }
193
194  private void addHole(RegionInfo a, RegionInfo b) {
195    this.report.holes.add(new Pair<>(a, b));
196  }
197
198  /** Returns True if table is disabled or disabling; defaults false! */
199  boolean isTableDisabled(RegionInfo ri) {
200    if (ri == null) {
201      return false;
202    }
203    if (this.services == null) {
204      return false;
205    }
206    if (this.services.getTableStateManager() == null) {
207      return false;
208    }
209    TableState state = null;
210    try {
211      state = this.services.getTableStateManager().getTableState(ri.getTable());
212    } catch (IOException e) {
213      LOG.warn("Failed getting table state", e);
214    }
215    return state != null && state.isDisabledOrDisabling();
216  }
217
218  /**
219   * Run through referenced servers and save off unknown and the dead.
220   */
221  private void checkServer(RegionLocations locations) {
222    if (this.services == null) {
223      // Can't do this test if no services.
224      return;
225    }
226    if (locations == null) {
227      return;
228    }
229    if (locations.getRegionLocations() == null) {
230      return;
231    }
232    // Check referenced servers are known/online. Here we are looking
233    // at both the default replica -- the main replica -- and then replica
234    // locations too.
235    for (HRegionLocation location : locations.getRegionLocations()) {
236      if (location == null) {
237        continue;
238      }
239      ServerName sn = location.getServerName();
240      if (sn == null) {
241        continue;
242      }
243      if (location.getRegion() == null) {
244        LOG.warn("Empty RegionInfo in {}", location);
245        // This should never happen but if it does, will mess up below.
246        continue;
247      }
248      RegionInfo ri = location.getRegion();
249      // Skip split parent region
250      if (ri.isSplitParent()) {
251        continue;
252      }
253      // skip the offline regions which belong to disabled table.
254      if (isTableDisabled(ri)) {
255        continue;
256      }
257      RegionState rs = this.services.getAssignmentManager().getRegionStates().getRegionState(ri);
258      if (rs == null || rs.isClosedOrAbnormallyClosed()) {
259        // If closed against an 'Unknown Server', that is should be fine.
260        continue;
261      }
262      ServerManager.ServerLiveState state =
263        this.services.getServerManager().isServerKnownAndOnline(sn);
264      switch (state) {
265        case UNKNOWN:
266          this.report.unknownServers.add(new Pair<>(ri, sn));
267          break;
268
269        default:
270          break;
271      }
272    }
273  }
274
275  /** Returns True iff first row in hbase:meta or if we've broached a new table in hbase:meta */
276  private boolean isTableTransition(RegionInfo ri) {
277    return this.previous == null || !this.previous.getTable().equals(ri.getTable());
278  }
279
280  @Override
281  public void close() throws IOException {
282    // This is a table transition... after the last region. Check previous.
283    // Should be last region. If not, its a hole on end of laster table.
284    if (this.previous != null && !this.previous.isLast()) {
285      addHole(this.previous, RegionInfoBuilder.UNDEFINED);
286    }
287    this.closed = true;
288  }
289}