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.http;
019
020import java.util.Collections;
021import java.util.LinkedHashMap;
022import java.util.List;
023import java.util.Map;
024import java.util.stream.Collectors;
025import java.util.stream.StreamSupport;
026import org.apache.commons.lang3.builder.EqualsBuilder;
027import org.apache.commons.lang3.builder.HashCodeBuilder;
028import org.apache.commons.lang3.builder.ToStringBuilder;
029import org.apache.commons.lang3.builder.ToStringStyle;
030import org.apache.hadoop.hbase.CatalogFamilyFormat;
031import org.apache.hadoop.hbase.HConstants;
032import org.apache.hadoop.hbase.HRegionLocation;
033import org.apache.hadoop.hbase.MetaTableAccessor;
034import org.apache.hadoop.hbase.RegionLocations;
035import org.apache.hadoop.hbase.ServerName;
036import org.apache.hadoop.hbase.client.RegionInfo;
037import org.apache.hadoop.hbase.client.Result;
038import org.apache.hadoop.hbase.master.RegionState;
039import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
040import org.apache.hadoop.hbase.util.Bytes;
041import org.apache.hadoop.hbase.util.PairOfSameType;
042import org.apache.yetus.audience.InterfaceAudience;
043
044/**
045 * A POJO that consolidates the information about a single region replica that's stored in meta.
046 */
047@InterfaceAudience.Private
048public final class RegionReplicaInfo {
049  private final byte[] row;
050  private final RegionInfo regionInfo;
051  private final RegionState.State regionState;
052  private final ServerName serverName;
053  private final long seqNum;
054  /** See {@link org.apache.hadoop.hbase.HConstants#SERVERNAME_QUALIFIER_STR}. */
055  private final ServerName targetServerName;
056  private final Map<String, RegionInfo> mergeRegionInfo;
057  private final Map<String, RegionInfo> splitRegionInfo;
058
059  private RegionReplicaInfo(final Result result, final HRegionLocation location) {
060    this.row = result != null ? result.getRow() : null;
061    this.regionInfo = location != null ? location.getRegion() : null;
062    this.regionState = (result != null && regionInfo != null)
063      ? RegionStateStore.getRegionState(result, regionInfo)
064      : null;
065    this.serverName = location != null ? location.getServerName() : null;
066    this.seqNum = (location != null) ? location.getSeqNum() : HConstants.NO_SEQNUM;
067    this.targetServerName = (result != null && regionInfo != null)
068      ? MetaTableAccessor.getTargetServerName(result, regionInfo.getReplicaId())
069      : null;
070    this.mergeRegionInfo =
071      (result != null) ? CatalogFamilyFormat.getMergeRegionsWithName(result.rawCells()) : null;
072
073    if (result != null) {
074      PairOfSameType<RegionInfo> daughterRegions = MetaTableAccessor.getDaughterRegions(result);
075      this.splitRegionInfo = new LinkedHashMap<>();
076      if (daughterRegions.getFirst() != null) {
077        splitRegionInfo.put(HConstants.SPLITA_QUALIFIER_STR, daughterRegions.getFirst());
078      }
079      if (daughterRegions.getSecond() != null) {
080        splitRegionInfo.put(HConstants.SPLITB_QUALIFIER_STR, daughterRegions.getSecond());
081      }
082    } else {
083      this.splitRegionInfo = null;
084    }
085  }
086
087  public static List<RegionReplicaInfo> from(final Result result) {
088    if (result == null) {
089      return Collections.singletonList(null);
090    }
091
092    final RegionLocations locations = CatalogFamilyFormat.getRegionLocations(result);
093    if (locations == null) {
094      return Collections.singletonList(null);
095    }
096
097    return StreamSupport.stream(locations.spliterator(), false)
098      .map(location -> new RegionReplicaInfo(result, location)).collect(Collectors.toList());
099  }
100
101  public byte[] getRow() {
102    return row;
103  }
104
105  public RegionInfo getRegionInfo() {
106    return regionInfo;
107  }
108
109  public byte[] getRegionName() {
110    return regionInfo != null ? regionInfo.getRegionName() : null;
111  }
112
113  public byte[] getStartKey() {
114    return regionInfo != null ? regionInfo.getStartKey() : null;
115  }
116
117  public byte[] getEndKey() {
118    return regionInfo != null ? regionInfo.getEndKey() : null;
119  }
120
121  public Integer getReplicaId() {
122    return regionInfo != null ? regionInfo.getReplicaId() : null;
123  }
124
125  public RegionState.State getRegionState() {
126    return regionState;
127  }
128
129  public ServerName getServerName() {
130    return serverName;
131  }
132
133  public long getSeqNum() {
134    return seqNum;
135  }
136
137  public ServerName getTargetServerName() {
138    return targetServerName;
139  }
140
141  public Map<String, RegionInfo> getMergeRegionInfo() {
142    return mergeRegionInfo;
143  }
144
145  public Map<String, RegionInfo> getSplitRegionInfo() {
146    return splitRegionInfo;
147  }
148
149  @Override
150  public boolean equals(Object other) {
151    if (this == other) {
152      return true;
153    }
154
155    if (other == null || getClass() != other.getClass()) {
156      return false;
157    }
158
159    RegionReplicaInfo that = (RegionReplicaInfo) other;
160
161    return new EqualsBuilder().append(row, that.row).append(regionInfo, that.regionInfo)
162      .append(regionState, that.regionState).append(serverName, that.serverName)
163      .append(seqNum, that.seqNum).append(targetServerName, that.targetServerName)
164      .append(mergeRegionInfo, that.mergeRegionInfo).append(splitRegionInfo, that.splitRegionInfo)
165      .isEquals();
166  }
167
168  @Override
169  public int hashCode() {
170    return new HashCodeBuilder(17, 37).append(row).append(regionInfo).append(regionState)
171      .append(serverName).append(seqNum).append(targetServerName).append(mergeRegionInfo)
172      .append(splitRegionInfo).toHashCode();
173  }
174
175  @Override
176  public String toString() {
177    return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
178      .append("row", Bytes.toStringBinary(row)).append("regionInfo", regionInfo)
179      .append("regionState", regionState).append("serverName", serverName).append("seqNum", seqNum)
180      .append("transitioningOnServerName", targetServerName).append("merge*", mergeRegionInfo)
181      .append("split*", splitRegionInfo).toString();
182  }
183}