1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.client;
20
21 import java.io.IOException;
22 import java.util.ArrayList;
23 import java.util.List;
24
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.hbase.HRegionInfo;
27 import org.apache.hadoop.hbase.HRegionLocation;
28 import org.apache.hadoop.hbase.RegionLocations;
29 import org.apache.hadoop.hbase.TableName;
30 import org.apache.hadoop.hbase.classification.InterfaceAudience;
31 import org.apache.hadoop.hbase.classification.InterfaceStability;
32 import org.apache.hadoop.hbase.util.Pair;
33
34 import com.google.common.annotations.VisibleForTesting;
35
36
37
38
39
40
41
42
43
44 @InterfaceAudience.Private
45 @InterfaceStability.Stable
46 public class HRegionLocator implements RegionLocator {
47
48 private final TableName tableName;
49 private final ClusterConnection connection;
50
51 public HRegionLocator(TableName tableName, ClusterConnection connection) {
52 this.connection = connection;
53 this.tableName = tableName;
54 }
55
56
57
58
59 @Override
60 public void close() throws IOException {
61
62
63 }
64
65
66
67
68 @Override
69 public HRegionLocation getRegionLocation(final byte [] row)
70 throws IOException {
71 return connection.getRegionLocation(tableName, row, false);
72 }
73
74
75
76
77 @Override
78 public HRegionLocation getRegionLocation(final byte [] row, boolean reload)
79 throws IOException {
80 return connection.getRegionLocation(tableName, row, reload);
81 }
82
83 @Override
84 public List<HRegionLocation> getAllRegionLocations() throws IOException {
85 TableName tableName = getName();
86 ArrayList<HRegionLocation> regions = new ArrayList<>();
87 for (RegionLocations locations : listRegionLocations()) {
88 for (HRegionLocation location : locations.getRegionLocations()) {
89 regions.add(location);
90 }
91 connection.cacheLocation(tableName, locations);
92 }
93 return regions;
94 }
95
96
97
98
99 @Override
100 public byte[][] getStartKeys() throws IOException {
101 return getStartEndKeys().getFirst();
102 }
103
104
105
106
107 @Override
108 public byte[][] getEndKeys() throws IOException {
109 return getStartEndKeys().getSecond();
110 }
111
112
113
114
115 @Override
116 public Pair<byte[][], byte[][]> getStartEndKeys() throws IOException {
117 return getStartEndKeys(listRegionLocations());
118 }
119
120 @VisibleForTesting
121 Pair<byte[][], byte[][]> getStartEndKeys(List<RegionLocations> regions) {
122 final byte[][] startKeyList = new byte[regions.size()][];
123 final byte[][] endKeyList = new byte[regions.size()][];
124
125 for (int i = 0; i < regions.size(); i++) {
126 HRegionInfo region = regions.get(i).getRegionLocation().getRegionInfo();
127 startKeyList[i] = region.getStartKey();
128 endKeyList[i] = region.getEndKey();
129 }
130
131 return new Pair<>(startKeyList, endKeyList);
132 }
133
134 @Override
135 public TableName getName() {
136 return this.tableName;
137 }
138
139 @VisibleForTesting
140 List<RegionLocations> listRegionLocations() throws IOException {
141 return MetaScanner.listTableRegionLocations(getConfiguration(), this.connection, getName());
142 }
143
144 public Configuration getConfiguration() {
145 return connection.getConfiguration();
146 }
147 }