1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.mapreduce;
21
22 import java.io.DataInput;
23 import java.io.DataOutput;
24 import java.io.IOException;
25 import java.util.Arrays;
26
27 import org.apache.hadoop.hbase.HConstants;
28 import org.apache.hadoop.hbase.util.Bytes;
29 import org.apache.hadoop.io.Writable;
30 import org.apache.hadoop.mapreduce.InputSplit;
31
32
33
34
35
36 public class TableSplit extends InputSplit
37 implements Writable, Comparable<TableSplit> {
38
39 private byte [] tableName;
40 private byte [] startRow;
41 private byte [] endRow;
42 private String regionLocation;
43
44
45 public TableSplit() {
46 this(HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY,
47 HConstants.EMPTY_BYTE_ARRAY, "");
48 }
49
50
51
52
53
54
55
56
57
58 public TableSplit(byte [] tableName, byte [] startRow, byte [] endRow,
59 final String location) {
60 this.tableName = tableName;
61 this.startRow = startRow;
62 this.endRow = endRow;
63 this.regionLocation = location;
64 }
65
66
67
68
69
70
71 public byte [] getTableName() {
72 return tableName;
73 }
74
75
76
77
78
79
80 public byte [] getStartRow() {
81 return startRow;
82 }
83
84
85
86
87
88
89 public byte [] getEndRow() {
90 return endRow;
91 }
92
93
94
95
96
97
98 public String getRegionLocation() {
99 return regionLocation;
100 }
101
102
103
104
105
106
107
108 @Override
109 public String[] getLocations() {
110 return new String[] {regionLocation};
111 }
112
113
114
115
116
117
118
119 @Override
120 public long getLength() {
121
122 return 0;
123 }
124
125
126
127
128
129
130
131 @Override
132 public void readFields(DataInput in) throws IOException {
133 tableName = Bytes.readByteArray(in);
134 startRow = Bytes.readByteArray(in);
135 endRow = Bytes.readByteArray(in);
136 regionLocation = Bytes.toString(Bytes.readByteArray(in));
137 }
138
139
140
141
142
143
144
145 @Override
146 public void write(DataOutput out) throws IOException {
147 Bytes.writeByteArray(out, tableName);
148 Bytes.writeByteArray(out, startRow);
149 Bytes.writeByteArray(out, endRow);
150 Bytes.writeByteArray(out, Bytes.toBytes(regionLocation));
151 }
152
153
154
155
156
157
158
159 @Override
160 public String toString() {
161 return regionLocation + ":" +
162 Bytes.toStringBinary(startRow) + "," + Bytes.toStringBinary(endRow);
163 }
164
165
166
167
168
169
170
171
172 @Override
173 public int compareTo(TableSplit split) {
174 return Bytes.compareTo(getStartRow(), split.getStartRow());
175 }
176
177 @Override
178 public boolean equals(Object o) {
179 if (o == null || !(o instanceof TableSplit)) {
180 return false;
181 }
182 return Bytes.equals(tableName, ((TableSplit)o).tableName) &&
183 Bytes.equals(startRow, ((TableSplit)o).startRow) &&
184 Bytes.equals(endRow, ((TableSplit)o).endRow) &&
185 regionLocation.equals(((TableSplit)o).regionLocation);
186 }
187
188 @Override
189 public int hashCode() {
190 int result = tableName != null ? Arrays.hashCode(tableName) : 0;
191 result = 31 * result + (startRow != null ? Arrays.hashCode(startRow) : 0);
192 result = 31 * result + (endRow != null ? Arrays.hashCode(endRow) : 0);
193 result = 31 * result + (regionLocation != null ? regionLocation.hashCode() : 0);
194 return result;
195 }
196 }