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
35
36
37
38
39
40
41
42 @InterfaceAudience.Private
43 @InterfaceStability.Stable
44 public class HRegionLocator implements RegionLocator {
45
46 private final TableName tableName;
47 private final ClusterConnection connection;
48
49 public HRegionLocator(TableName tableName, ClusterConnection connection) {
50 this.connection = connection;
51 this.tableName = tableName;
52 }
53
54
55
56
57 @Override
58 public void close() throws IOException {
59
60
61 }
62
63
64
65
66 @Override
67 public HRegionLocation getRegionLocation(final byte [] row)
68 throws IOException {
69 return connection.getRegionLocation(tableName, row, false);
70 }
71
72
73
74
75 @Override
76 public HRegionLocation getRegionLocation(final byte [] row, boolean reload)
77 throws IOException {
78 return connection.getRegionLocation(tableName, row, reload);
79 }
80
81 @Override
82 public List<HRegionLocation> getAllRegionLocations() throws IOException {
83 TableName tableName = getName();
84 ArrayList<HRegionLocation> regions = new ArrayList<>();
85 for (RegionLocations locations : listRegionLocations()) {
86 for (HRegionLocation location : locations.getRegionLocations()) {
87 regions.add(location);
88 }
89 connection.cacheLocation(tableName, locations);
90 }
91 return regions;
92 }
93
94
95
96
97 @Override
98 public byte[][] getStartKeys() throws IOException {
99 return getStartEndKeys().getFirst();
100 }
101
102
103
104
105 @Override
106 public byte[][] getEndKeys() throws IOException {
107 return getStartEndKeys().getSecond();
108 }
109
110
111
112
113 @Override
114 public Pair<byte[][], byte[][]> getStartEndKeys() throws IOException {
115 return getStartEndKeys(listRegionLocations());
116 }
117
118 Pair<byte[][], byte[][]> getStartEndKeys(List<RegionLocations> regions) {
119 final byte[][] startKeyList = new byte[regions.size()][];
120 final byte[][] endKeyList = new byte[regions.size()][];
121
122 for (int i = 0; i < regions.size(); i++) {
123 HRegionInfo region = regions.get(i).getRegionLocation().getRegionInfo();
124 startKeyList[i] = region.getStartKey();
125 endKeyList[i] = region.getEndKey();
126 }
127
128 return new Pair<>(startKeyList, endKeyList);
129 }
130
131 @Override
132 public TableName getName() {
133 return this.tableName;
134 }
135
136 List<RegionLocations> listRegionLocations() throws IOException {
137 return MetaScanner.listTableRegionLocations(getConfiguration(), this.connection, getName());
138 }
139
140 public Configuration getConfiguration() {
141 return connection.getConfiguration();
142 }
143 }