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.io.IOException;
021import java.util.ArrayList;
022import java.util.Collections;
023import java.util.List;
024import java.util.SortedMap;
025import java.util.TreeMap;
026import org.apache.hadoop.hbase.Cell;
027import org.apache.hadoop.hbase.CellBuilderFactory;
028import org.apache.hadoop.hbase.CellBuilderType;
029import org.apache.hadoop.hbase.HConstants;
030import org.apache.hadoop.hbase.HRegionLocation;
031import org.apache.hadoop.hbase.MetaTableAccessor;
032import org.apache.hadoop.hbase.RegionLocations;
033import org.apache.hadoop.hbase.ServerName;
034import org.apache.hadoop.hbase.TableName;
035import org.apache.hadoop.hbase.client.Delete;
036import org.apache.hadoop.hbase.client.Mutation;
037import org.apache.hadoop.hbase.client.Put;
038import org.apache.hadoop.hbase.client.RegionInfo;
039import org.apache.hadoop.hbase.client.Result;
040import org.apache.hadoop.hbase.client.ResultScanner;
041import org.apache.hadoop.hbase.client.Scan;
042import org.apache.hadoop.hbase.client.Table;
043import org.apache.hadoop.hbase.client.TableDescriptor;
044import org.apache.hadoop.hbase.master.MasterFileSystem;
045import org.apache.hadoop.hbase.master.MasterServices;
046import org.apache.hadoop.hbase.master.RegionState.State;
047import org.apache.hadoop.hbase.master.region.MasterRegion;
048import org.apache.hadoop.hbase.procedure2.Procedure;
049import org.apache.hadoop.hbase.procedure2.util.StringUtils;
050import org.apache.hadoop.hbase.util.Bytes;
051import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
052import org.apache.hadoop.hbase.wal.WALSplitUtil;
053import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
054import org.apache.yetus.audience.InterfaceAudience;
055import org.apache.zookeeper.KeeperException;
056import org.slf4j.Logger;
057import org.slf4j.LoggerFactory;
058
059import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
060
061/**
062 * Store Region State to hbase:meta table.
063 */
064@InterfaceAudience.Private
065public class RegionStateStore {
066  private static final Logger LOG = LoggerFactory.getLogger(RegionStateStore.class);
067  private static final Logger METALOG = LoggerFactory.getLogger("org.apache.hadoop.hbase.META");
068
069  /** The delimiter for meta columns for replicaIds > 0 */
070  protected static final char META_REPLICA_ID_DELIMITER = '_';
071
072  private final MasterServices master;
073
074  private final MasterRegion masterRegion;
075
076  public RegionStateStore(MasterServices master, MasterRegion masterRegion) {
077    this.master = master;
078    this.masterRegion = masterRegion;
079  }
080
081  @FunctionalInterface
082  public interface RegionStateVisitor {
083    void visitRegionState(Result result, RegionInfo regionInfo, State state,
084      ServerName regionLocation, ServerName lastHost, long openSeqNum);
085  }
086
087  public void visitMeta(final RegionStateVisitor visitor) throws IOException {
088    MetaTableAccessor.fullScanRegions(master.getConnection(), new MetaTableAccessor.Visitor() {
089      final boolean isDebugEnabled = LOG.isDebugEnabled();
090
091      @Override
092      public boolean visit(final Result r) throws IOException {
093        if (r != null && !r.isEmpty()) {
094          long st = 0;
095          if (LOG.isTraceEnabled()) {
096            st = EnvironmentEdgeManager.currentTime();
097          }
098          visitMetaEntry(visitor, r);
099          if (LOG.isTraceEnabled()) {
100            long et = EnvironmentEdgeManager.currentTime();
101            LOG.trace("[T] LOAD META PERF " + StringUtils.humanTimeDiff(et - st));
102          }
103        } else if (isDebugEnabled) {
104          LOG.debug("NULL result from meta - ignoring but this is strange.");
105        }
106        return true;
107      }
108    });
109  }
110
111  /**
112   * Queries META table for the passed region encoded name, delegating action upon results to the
113   * <code>RegionStateVisitor</code> passed as second parameter.
114   * @param regionEncodedName encoded name for the Region we want to query META for.
115   * @param visitor           The <code>RegionStateVisitor</code> instance to react over the query
116   *                          results.
117   * @throws IOException If some error occurs while querying META or parsing results.
118   */
119  public void visitMetaForRegion(final String regionEncodedName, final RegionStateVisitor visitor)
120    throws IOException {
121    Result result =
122      MetaTableAccessor.scanByRegionEncodedName(master.getConnection(), regionEncodedName);
123    if (result != null) {
124      visitMetaEntry(visitor, result);
125    }
126  }
127
128  public static void visitMetaEntry(final RegionStateVisitor visitor, final Result result)
129    throws IOException {
130    final RegionLocations rl = MetaTableAccessor.getRegionLocations(result);
131    if (rl == null) return;
132
133    final HRegionLocation[] locations = rl.getRegionLocations();
134    if (locations == null) return;
135
136    for (int i = 0; i < locations.length; ++i) {
137      final HRegionLocation hrl = locations[i];
138      if (hrl == null) continue;
139
140      final RegionInfo regionInfo = hrl.getRegion();
141      if (regionInfo == null) continue;
142
143      final int replicaId = regionInfo.getReplicaId();
144      final State state = getRegionState(result, regionInfo);
145
146      final ServerName lastHost = hrl.getServerName();
147      ServerName regionLocation = MetaTableAccessor.getTargetServerName(result, replicaId);
148      final long openSeqNum = hrl.getSeqNum();
149
150      LOG.debug(
151        "Load hbase:meta entry region={}, regionState={}, lastHost={}, "
152          + "regionLocation={}, openSeqNum={}",
153        regionInfo.getEncodedName(), state, lastHost, regionLocation, openSeqNum);
154      visitor.visitRegionState(result, regionInfo, state, regionLocation, lastHost, openSeqNum);
155    }
156  }
157
158  void updateRegionLocation(RegionStateNode regionStateNode) throws IOException {
159    long time = EnvironmentEdgeManager.currentTime();
160    long openSeqNum = regionStateNode.getState() == State.OPEN
161      ? regionStateNode.getOpenSeqNum()
162      : HConstants.NO_SEQNUM;
163    RegionInfo regionInfo = regionStateNode.getRegionInfo();
164    State state = regionStateNode.getState();
165    ServerName regionLocation = regionStateNode.getRegionLocation();
166    TransitRegionStateProcedure rit = regionStateNode.getProcedure();
167    long pid = rit != null ? rit.getProcId() : Procedure.NO_PROC_ID;
168    final int replicaId = regionInfo.getReplicaId();
169    final Put put = new Put(MetaTableAccessor.getMetaKeyForRegion(regionInfo), time);
170    MetaTableAccessor.addRegionInfo(put, regionInfo);
171    final StringBuilder info =
172      new StringBuilder("pid=").append(pid).append(" updating hbase:meta row=")
173        .append(regionInfo.getEncodedName()).append(", regionState=").append(state);
174    if (openSeqNum >= 0) {
175      Preconditions.checkArgument(state == State.OPEN && regionLocation != null,
176        "Open region should be on a server");
177      MetaTableAccessor.addLocation(put, regionLocation, openSeqNum, replicaId);
178      // only update replication barrier for default replica
179      if (
180        regionInfo.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID
181          && hasGlobalReplicationScope(regionInfo.getTable())
182      ) {
183        MetaTableAccessor.addReplicationBarrier(put, openSeqNum);
184        info.append(", repBarrier=").append(openSeqNum);
185      }
186      info.append(", openSeqNum=").append(openSeqNum);
187      info.append(", regionLocation=").append(regionLocation);
188    } else if (regionLocation != null) {
189      // Ideally, if no regionLocation, write null to the hbase:meta but this will confuse clients
190      // currently; they want a server to hit. TODO: Make clients wait if no location.
191      put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
192        .setFamily(HConstants.CATALOG_FAMILY)
193        .setQualifier(MetaTableAccessor.getServerNameColumn(replicaId))
194        .setTimestamp(put.getTimestamp()).setType(Cell.Type.Put)
195        .setValue(Bytes.toBytes(regionLocation.getServerName())).build());
196      info.append(", regionLocation=").append(regionLocation);
197    }
198    put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
199      .setFamily(HConstants.CATALOG_FAMILY).setQualifier(getStateColumn(replicaId))
200      .setTimestamp(put.getTimestamp()).setType(Cell.Type.Put).setValue(Bytes.toBytes(state.name()))
201      .build());
202    LOG.info(info.toString());
203    updateRegionLocation(regionInfo, state, put);
204    if (regionInfo.isMetaRegion() && regionInfo.isFirst()) {
205      // mirror the meta location to zookeeper
206      mirrorMetaLocation(regionInfo, regionLocation, state);
207    }
208  }
209
210  private void mirrorMetaLocation(RegionInfo regionInfo, ServerName serverName, State state)
211    throws IOException {
212    try {
213      MetaTableLocator.setMetaLocation(master.getZooKeeper(), serverName, regionInfo.getReplicaId(),
214        state);
215    } catch (KeeperException e) {
216      throw new IOException(e);
217    }
218  }
219
220  private void removeMirrorMetaLocation(int oldReplicaCount, int newReplicaCount)
221    throws IOException {
222    try {
223      for (int i = newReplicaCount; i < oldReplicaCount; i++) {
224        MetaTableLocator.deleteMetaLocation(master.getZooKeeper(), i);
225      }
226    } catch (KeeperException e) {
227      throw new IOException(e);
228    }
229  }
230
231  private void updateRegionLocation(RegionInfo regionInfo, State state, Put put)
232    throws IOException {
233    try {
234      if (regionInfo.isMetaRegion()) {
235        masterRegion.update(r -> r.put(put));
236      } else {
237        try (Table table = master.getConnection().getTable(TableName.META_TABLE_NAME)) {
238          table.put(put);
239        }
240      }
241    } catch (IOException e) {
242      // TODO: Revist!!!! Means that if a server is loaded, then we will abort our host!
243      // In tests we abort the Master!
244      String msg = String.format("FAILED persisting region=%s state=%s",
245        regionInfo.getShortNameToLog(), state);
246      LOG.error(msg, e);
247      master.abort(msg, e);
248      throw e;
249    }
250  }
251
252  private long getOpenSeqNumForParentRegion(RegionInfo region) throws IOException {
253    MasterFileSystem fs = master.getMasterFileSystem();
254    long maxSeqId = WALSplitUtil.getMaxRegionSequenceId(master.getConfiguration(), region,
255      fs::getFileSystem, fs::getWALFileSystem);
256    return maxSeqId > 0 ? maxSeqId + 1 : HConstants.NO_SEQNUM;
257  }
258
259  private Table getMetaTable() throws IOException {
260    return master.getConnection().getTable(TableName.META_TABLE_NAME);
261  }
262
263  // ============================================================================================
264  // Update Region Splitting State helpers
265  // ============================================================================================
266  public void splitRegion(RegionInfo parent, RegionInfo hriA, RegionInfo hriB,
267    ServerName serverName) throws IOException {
268    TableDescriptor htd = getTableDescriptor(parent.getTable());
269    long parentOpenSeqNum = HConstants.NO_SEQNUM;
270    if (htd.hasGlobalReplicationScope()) {
271      parentOpenSeqNum = getOpenSeqNumForParentRegion(parent);
272    }
273    MetaTableAccessor.splitRegion(master.getConnection(), parent, parentOpenSeqNum, hriA, hriB,
274      serverName, getRegionReplication(htd));
275  }
276
277  // ============================================================================================
278  // Update Region Merging State helpers
279  // ============================================================================================
280  public void mergeRegions(RegionInfo child, RegionInfo[] parents, ServerName serverName)
281    throws IOException {
282    TableDescriptor htd = getTableDescriptor(child.getTable());
283    boolean globalScope = htd.hasGlobalReplicationScope();
284    SortedMap<RegionInfo, Long> parentSeqNums = new TreeMap<>();
285    for (RegionInfo ri : parents) {
286      parentSeqNums.put(ri, globalScope ? getOpenSeqNumForParentRegion(ri) : -1);
287    }
288    MetaTableAccessor.mergeRegions(master.getConnection(), child, parentSeqNums, serverName,
289      getRegionReplication(htd));
290  }
291
292  // ============================================================================================
293  // Delete Region State helpers
294  // ============================================================================================
295  public void deleteRegion(final RegionInfo regionInfo) throws IOException {
296    deleteRegions(Collections.singletonList(regionInfo));
297  }
298
299  public void deleteRegions(final List<RegionInfo> regions) throws IOException {
300    MetaTableAccessor.deleteRegionInfos(master.getConnection(), regions);
301  }
302
303  private Scan getScanForUpdateRegionReplicas(TableName tableName) {
304    Scan scan;
305    if (TableName.isMetaTableName(tableName)) {
306      // Notice that, we do not use MetaCellComparator for master local region, so we can not use
307      // the same logic to set start key and end key for scanning meta table when locating entries
308      // in master local region. And since there is only one table in master local region(the record
309      // for meta table), so we do not need set start key and end key.
310      scan = new Scan();
311    } else {
312      scan = MetaTableAccessor.getScanForTableName(master.getConfiguration(), tableName);
313    }
314    return scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
315  }
316
317  private List<Delete> deleteRegionReplicas(ResultScanner scanner, int oldReplicaCount,
318    int newReplicaCount, long now) throws IOException {
319    List<Delete> deletes = new ArrayList<>();
320    for (;;) {
321      Result result = scanner.next();
322      if (result == null) {
323        break;
324      }
325      RegionInfo primaryRegionInfo = MetaTableAccessor.getRegionInfo(result);
326      if (primaryRegionInfo == null || primaryRegionInfo.isSplit()) {
327        continue;
328      }
329      Delete delete = new Delete(result.getRow());
330      for (int i = newReplicaCount; i < oldReplicaCount; i++) {
331        delete.addColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerColumn(i), now);
332        delete.addColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getSeqNumColumn(i), now);
333        delete.addColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getStartCodeColumn(i), now);
334        delete.addColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerNameColumn(i), now);
335        delete.addColumns(HConstants.CATALOG_FAMILY, MetaTableAccessor.getRegionStateColumn(i),
336          now);
337      }
338      deletes.add(delete);
339    }
340    return deletes;
341  }
342
343  public void removeRegionReplicas(TableName tableName, int oldReplicaCount, int newReplicaCount)
344    throws IOException {
345    Scan scan = getScanForUpdateRegionReplicas(tableName);
346    long now = EnvironmentEdgeManager.currentTime();
347    if (TableName.isMetaTableName(tableName)) {
348      List<Delete> deletes;
349      try (ResultScanner scanner = masterRegion.getScanner(scan)) {
350        deletes = deleteRegionReplicas(scanner, oldReplicaCount, newReplicaCount, now);
351      }
352      debugLogMutations(deletes);
353      masterRegion.update(r -> {
354        for (Delete d : deletes) {
355          r.delete(d);
356        }
357      });
358      // also delete the mirrored location on zk
359      removeMirrorMetaLocation(oldReplicaCount, newReplicaCount);
360    } else {
361      try (Table metaTable = getMetaTable(); ResultScanner scanner = metaTable.getScanner(scan)) {
362        List<Delete> deletes = deleteRegionReplicas(scanner, oldReplicaCount, newReplicaCount, now);
363        debugLogMutations(deletes);
364        metaTable.delete(deletes);
365      }
366    }
367  }
368
369  // ==========================================================================
370  // Table Descriptors helpers
371  // ==========================================================================
372  private boolean hasGlobalReplicationScope(TableName tableName) throws IOException {
373    return hasGlobalReplicationScope(getTableDescriptor(tableName));
374  }
375
376  private boolean hasGlobalReplicationScope(TableDescriptor htd) {
377    return htd != null ? htd.hasGlobalReplicationScope() : false;
378  }
379
380  private int getRegionReplication(TableDescriptor htd) {
381    return htd != null ? htd.getRegionReplication() : 1;
382  }
383
384  private TableDescriptor getTableDescriptor(TableName tableName) throws IOException {
385    return master.getTableDescriptors().get(tableName);
386  }
387
388  // ==========================================================================
389  // Region State
390  // ==========================================================================
391
392  /**
393   * Pull the region state from a catalog table {@link Result}.
394   * @return the region state, or null if unknown.
395   */
396  public static State getRegionState(final Result r, RegionInfo regionInfo) {
397    Cell cell =
398      r.getColumnLatestCell(HConstants.CATALOG_FAMILY, getStateColumn(regionInfo.getReplicaId()));
399    if (cell == null || cell.getValueLength() == 0) {
400      return null;
401    }
402
403    String state =
404      Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
405    try {
406      return State.valueOf(state);
407    } catch (IllegalArgumentException e) {
408      LOG.warn(
409        "BAD value {} in hbase:meta info:state column for region {} , "
410          + "Consider using HBCK2 setRegionState ENCODED_REGION_NAME STATE",
411        state, regionInfo.getEncodedName());
412      return null;
413    }
414  }
415
416  public static byte[] getStateColumn(int replicaId) {
417    return replicaId == 0
418      ? HConstants.STATE_QUALIFIER
419      : Bytes.toBytes(HConstants.STATE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
420        + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
421  }
422
423  private static void debugLogMutations(List<? extends Mutation> mutations) throws IOException {
424    if (!METALOG.isDebugEnabled()) {
425      return;
426    }
427    // Logging each mutation in separate line makes it easier to see diff between them visually
428    // because of common starting indentation.
429    for (Mutation mutation : mutations) {
430      debugLogMutation(mutation);
431    }
432  }
433
434  private static void debugLogMutation(Mutation p) throws IOException {
435    METALOG.debug("{} {}", p.getClass().getSimpleName(), p.toJSON());
436  }
437}