001/** 002 * 003 * Licensed to the Apache Software Foundation (ASF) under one 004 * or more contributor license agreements. See the NOTICE file 005 * distributed with this work for additional information 006 * regarding copyright ownership. The ASF licenses this file 007 * to you under the Apache License, Version 2.0 (the 008 * "License"); you may not use this file except in compliance 009 * with the License. You may obtain a copy of the License at 010 * 011 * http://www.apache.org/licenses/LICENSE-2.0 012 * 013 * Unless required by applicable law or agreed to in writing, software 014 * distributed under the License is distributed on an "AS IS" BASIS, 015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 016 * See the License for the specific language governing permissions and 017 * limitations under the License. 018 */ 019 020package org.apache.hadoop.hbase.master.assignment; 021 022import java.io.IOException; 023import java.util.Collections; 024import java.util.List; 025 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.Put; 036import org.apache.hadoop.hbase.client.RegionInfo; 037import org.apache.hadoop.hbase.client.Result; 038import org.apache.hadoop.hbase.client.Table; 039import org.apache.hadoop.hbase.client.TableDescriptor; 040import org.apache.hadoop.hbase.master.MasterServices; 041import org.apache.hadoop.hbase.master.RegionState.State; 042import org.apache.hadoop.hbase.procedure2.Procedure; 043import org.apache.hadoop.hbase.procedure2.util.StringUtils; 044import org.apache.hadoop.hbase.util.Bytes; 045import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 046import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; 047import org.apache.yetus.audience.InterfaceAudience; 048import org.apache.zookeeper.KeeperException; 049import org.slf4j.Logger; 050import org.slf4j.LoggerFactory; 051 052import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; 053import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; 054 055/** 056 * Store Region State to hbase:meta table. 057 */ 058@InterfaceAudience.Private 059public class RegionStateStore { 060 private static final Logger LOG = LoggerFactory.getLogger(RegionStateStore.class); 061 062 /** The delimiter for meta columns for replicaIds > 0 */ 063 protected static final char META_REPLICA_ID_DELIMITER = '_'; 064 065 private final MasterServices master; 066 067 public RegionStateStore(final MasterServices master) { 068 this.master = master; 069 } 070 071 public interface RegionStateVisitor { 072 void visitRegionState(Result result, RegionInfo regionInfo, State state, 073 ServerName regionLocation, ServerName lastHost, long openSeqNum); 074 } 075 076 public void visitMeta(final RegionStateVisitor visitor) throws IOException { 077 MetaTableAccessor.fullScanRegions(master.getConnection(), new MetaTableAccessor.Visitor() { 078 final boolean isDebugEnabled = LOG.isDebugEnabled(); 079 080 @Override 081 public boolean visit(final Result r) throws IOException { 082 if (r != null && !r.isEmpty()) { 083 long st = 0; 084 if (LOG.isTraceEnabled()) { 085 st = System.currentTimeMillis(); 086 } 087 visitMetaEntry(visitor, r); 088 if (LOG.isTraceEnabled()) { 089 long et = System.currentTimeMillis(); 090 LOG.trace("[T] LOAD META PERF " + StringUtils.humanTimeDiff(et - st)); 091 } 092 } else if (isDebugEnabled) { 093 LOG.debug("NULL result from meta - ignoring but this is strange."); 094 } 095 return true; 096 } 097 }); 098 } 099 100 private void visitMetaEntry(final RegionStateVisitor visitor, final Result result) 101 throws IOException { 102 final RegionLocations rl = MetaTableAccessor.getRegionLocations(result); 103 if (rl == null) return; 104 105 final HRegionLocation[] locations = rl.getRegionLocations(); 106 if (locations == null) return; 107 108 for (int i = 0; i < locations.length; ++i) { 109 final HRegionLocation hrl = locations[i]; 110 if (hrl == null) continue; 111 112 final RegionInfo regionInfo = hrl.getRegion(); 113 if (regionInfo == null) continue; 114 115 final int replicaId = regionInfo.getReplicaId(); 116 final State state = getRegionState(result, replicaId); 117 118 final ServerName lastHost = hrl.getServerName(); 119 final ServerName regionLocation = getRegionServer(result, replicaId); 120 final long openSeqNum = hrl.getSeqNum(); 121 122 // TODO: move under trace, now is visible for debugging 123 LOG.info( 124 "Load hbase:meta entry region={}, regionState={}, lastHost={}, " + 125 "regionLocation={}, openSeqNum={}", 126 regionInfo.getEncodedName(), state, lastHost, regionLocation, openSeqNum); 127 visitor.visitRegionState(result, regionInfo, state, regionLocation, lastHost, openSeqNum); 128 } 129 } 130 131 public void updateRegionLocation(RegionStates.RegionStateNode regionStateNode) 132 throws IOException { 133 if (regionStateNode.getRegionInfo().isMetaRegion()) { 134 updateMetaLocation(regionStateNode.getRegionInfo(), regionStateNode.getRegionLocation(), 135 regionStateNode.getState()); 136 } else { 137 long openSeqNum = regionStateNode.getState() == State.OPEN ? regionStateNode.getOpenSeqNum() 138 : HConstants.NO_SEQNUM; 139 updateUserRegionLocation(regionStateNode.getRegionInfo(), regionStateNode.getState(), 140 regionStateNode.getRegionLocation(), openSeqNum, 141 // The regionStateNode may have no procedure in a test scenario; allow for this. 142 regionStateNode.getProcedure() != null ? regionStateNode.getProcedure().getProcId() 143 : Procedure.NO_PROC_ID); 144 } 145 } 146 147 private void updateMetaLocation(RegionInfo regionInfo, ServerName serverName, State state) 148 throws IOException { 149 try { 150 MetaTableLocator.setMetaLocation(master.getZooKeeper(), serverName, regionInfo.getReplicaId(), 151 state); 152 } catch (KeeperException e) { 153 throw new IOException(e); 154 } 155 } 156 157 private void updateUserRegionLocation(final RegionInfo regionInfo, final State state, 158 final ServerName regionLocation, final long openSeqNum, 159 final long pid) 160 throws IOException { 161 long time = EnvironmentEdgeManager.currentTime(); 162 final int replicaId = regionInfo.getReplicaId(); 163 final Put put = new Put(MetaTableAccessor.getMetaKeyForRegion(regionInfo), time); 164 MetaTableAccessor.addRegionInfo(put, regionInfo); 165 final StringBuilder info = 166 new StringBuilder("pid=").append(pid).append(" updating hbase:meta row=") 167 .append(regionInfo.getEncodedName()).append(", regionState=").append(state); 168 if (openSeqNum >= 0) { 169 Preconditions.checkArgument(state == State.OPEN && regionLocation != null, 170 "Open region should be on a server"); 171 MetaTableAccessor.addLocation(put, regionLocation, openSeqNum, replicaId); 172 info.append(", openSeqNum=").append(openSeqNum); 173 info.append(", regionLocation=").append(regionLocation); 174 } else if (regionLocation != null) { 175 // Ideally, if no regionLocation, write null to the hbase:meta but this will confuse clients 176 // currently; they want a server to hit. TODO: Make clients wait if no location. 177 put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY) 178 .setRow(put.getRow()) 179 .setFamily(HConstants.CATALOG_FAMILY) 180 .setQualifier(getServerNameColumn(replicaId)) 181 .setTimestamp(put.getTimestamp()) 182 .setType(Cell.Type.Put) 183 .setValue(Bytes.toBytes(regionLocation.getServerName())) 184 .build()); 185 info.append(", regionLocation=").append(regionLocation); 186 } 187 put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY) 188 .setRow(put.getRow()) 189 .setFamily(HConstants.CATALOG_FAMILY) 190 .setQualifier(getStateColumn(replicaId)) 191 .setTimestamp(put.getTimestamp()) 192 .setType(Cell.Type.Put) 193 .setValue(Bytes.toBytes(state.name())) 194 .build()); 195 LOG.info(info.toString()); 196 updateRegionLocation(regionInfo, state, put); 197 } 198 199 private void updateRegionLocation(RegionInfo regionInfo, State state, Put put) 200 throws IOException { 201 try (Table table = master.getConnection().getTable(TableName.META_TABLE_NAME)) { 202 table.put(put); 203 } catch (IOException e) { 204 // TODO: Revist!!!! Means that if a server is loaded, then we will abort our host! 205 // In tests we abort the Master! 206 String msg = String.format("FAILED persisting region=%s state=%s", 207 regionInfo.getShortNameToLog(), state); 208 LOG.error(msg, e); 209 master.abort(msg, e); 210 throw e; 211 } 212 } 213 214 // ============================================================================================ 215 // Update Region Splitting State helpers 216 // ============================================================================================ 217 public void splitRegion(final RegionInfo parent, final RegionInfo hriA, 218 final RegionInfo hriB, final ServerName serverName) throws IOException { 219 final TableDescriptor htd = getTableDescriptor(parent.getTable()); 220 MetaTableAccessor.splitRegion(master.getConnection(), parent, hriA, hriB, serverName, 221 getRegionReplication(htd)); 222 } 223 224 // ============================================================================================ 225 // Update Region Merging State helpers 226 // ============================================================================================ 227 public void mergeRegions(RegionInfo child, RegionInfo [] parents, ServerName serverName) 228 throws IOException { 229 TableDescriptor htd = getTableDescriptor(child.getTable()); 230 MetaTableAccessor.mergeRegions(master.getConnection(), child, parents, serverName, 231 getRegionReplication(htd)); 232 } 233 234 // ============================================================================================ 235 // Delete Region State helpers 236 // ============================================================================================ 237 public void deleteRegion(final RegionInfo regionInfo) throws IOException { 238 deleteRegions(Collections.singletonList(regionInfo)); 239 } 240 241 public void deleteRegions(final List<RegionInfo> regions) throws IOException { 242 MetaTableAccessor.deleteRegionInfos(master.getConnection(), regions); 243 } 244 245 // ========================================================================== 246 // Table Descriptors helpers 247 // ========================================================================== 248 private int getRegionReplication(final TableDescriptor htd) { 249 return (htd != null) ? htd.getRegionReplication() : 1; 250 } 251 252 private TableDescriptor getTableDescriptor(final TableName tableName) throws IOException { 253 return master.getTableDescriptors().get(tableName); 254 } 255 256 // ========================================================================== 257 // Server Name 258 // ========================================================================== 259 260 /** 261 * Returns the {@link ServerName} from catalog table {@link Result} 262 * where the region is transitioning. It should be the same as 263 * {@link MetaTableAccessor#getServerName(Result,int)} if the server is at OPEN state. 264 * @param r Result to pull the transitioning server name from 265 * @return A ServerName instance or {@link MetaTableAccessor#getServerName(Result,int)} 266 * if necessary fields not found or empty. 267 */ 268 static ServerName getRegionServer(final Result r, int replicaId) { 269 final Cell cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY, 270 getServerNameColumn(replicaId)); 271 if (cell == null || cell.getValueLength() == 0) { 272 RegionLocations locations = MetaTableAccessor.getRegionLocations(r); 273 if (locations != null) { 274 HRegionLocation location = locations.getRegionLocation(replicaId); 275 if (location != null) { 276 return location.getServerName(); 277 } 278 } 279 return null; 280 } 281 return ServerName.parseServerName(Bytes.toString(cell.getValueArray(), 282 cell.getValueOffset(), cell.getValueLength())); 283 } 284 285 private static byte[] getServerNameColumn(int replicaId) { 286 return replicaId == 0 287 ? HConstants.SERVERNAME_QUALIFIER 288 : Bytes.toBytes(HConstants.SERVERNAME_QUALIFIER_STR + META_REPLICA_ID_DELIMITER 289 + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId)); 290 } 291 292 // ========================================================================== 293 // Region State 294 // ========================================================================== 295 296 /** 297 * Pull the region state from a catalog table {@link Result}. 298 * @param r Result to pull the region state from 299 * @return the region state, or null if unknown. 300 */ 301 @VisibleForTesting 302 public static State getRegionState(final Result r, int replicaId) { 303 Cell cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY, getStateColumn(replicaId)); 304 if (cell == null || cell.getValueLength() == 0) { 305 return null; 306 } 307 return State.valueOf(Bytes.toString(cell.getValueArray(), cell.getValueOffset(), 308 cell.getValueLength())); 309 } 310 311 private static byte[] getStateColumn(int replicaId) { 312 return replicaId == 0 313 ? HConstants.STATE_QUALIFIER 314 : Bytes.toBytes(HConstants.STATE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER 315 + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId)); 316 } 317}