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