1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.io;
20
21 import java.io.BufferedInputStream;
22 import java.io.DataInput;
23 import java.io.DataInputStream;
24 import java.io.IOException;
25 import java.io.InputStream;
26
27 import org.apache.hadoop.classification.InterfaceAudience;
28 import org.apache.hadoop.fs.FSDataOutputStream;
29 import org.apache.hadoop.fs.FileSystem;
30 import org.apache.hadoop.fs.Path;
31 import org.apache.hadoop.hbase.KeyValue;
32 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
33 import org.apache.hadoop.hbase.protobuf.generated.FSProtos;
34 import org.apache.hadoop.hbase.util.Bytes;
35
36 import com.google.protobuf.ByteString;
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56 @InterfaceAudience.Private
57 public class Reference {
58 private byte [] splitkey;
59 private Range region;
60
61
62
63
64
65 static enum Range {
66
67 top,
68
69 bottom
70 }
71
72
73
74
75
76 public static Reference createTopReference(final byte [] splitRow) {
77 return new Reference(splitRow, Range.top);
78 }
79
80
81
82
83
84 public static Reference createBottomReference(final byte [] splitRow) {
85 return new Reference(splitRow, Range.bottom);
86 }
87
88
89
90
91
92
93 Reference(final byte [] splitRow, final Range fr) {
94 this.splitkey = splitRow == null? null: KeyValue.createFirstOnRow(splitRow).getKey();
95 this.region = fr;
96 }
97
98
99
100
101 @Deprecated
102
103 public Reference() {
104 this(null, Range.bottom);
105 }
106
107
108
109
110
111 public Range getFileRegion() {
112 return this.region;
113 }
114
115
116
117
118 public byte [] getSplitKey() {
119 return splitkey;
120 }
121
122
123
124
125 @Override
126 public String toString() {
127 return "" + this.region;
128 }
129
130 public static boolean isTopFileRegion(final Range r) {
131 return r.equals(Range.top);
132 }
133
134
135
136
137
138
139 @Deprecated
140 public void readFields(DataInput in) throws IOException {
141 boolean tmp = in.readBoolean();
142
143 this.region = tmp? Range.top: Range.bottom;
144 this.splitkey = Bytes.readByteArray(in);
145 }
146
147 public Path write(final FileSystem fs, final Path p)
148 throws IOException {
149 FSDataOutputStream out = fs.create(p, false);
150 try {
151 out.write(toByteArray());
152 } finally {
153 out.close();
154 }
155 return p;
156 }
157
158
159
160
161
162
163
164
165 public static Reference read(final FileSystem fs, final Path p)
166 throws IOException {
167 InputStream in = fs.open(p);
168 try {
169
170
171 in = in.markSupported()? in: new BufferedInputStream(in);
172 int pblen = ProtobufUtil.lengthOfPBMagic();
173 in.mark(pblen);
174 byte [] pbuf = new byte[pblen];
175 int read = in.read(pbuf);
176 if (read != pblen) throw new IOException("read=" + read + ", wanted=" + pblen);
177
178 if (ProtobufUtil.isPBMagicPrefix(pbuf)) return convert(FSProtos.Reference.parseFrom(in));
179
180
181 in.reset();
182 Reference r = new Reference();
183 DataInputStream dis = new DataInputStream(in);
184
185 in = dis;
186 r.readFields(dis);
187 return r;
188 } finally {
189 in.close();
190 }
191 }
192
193 FSProtos.Reference convert() {
194 FSProtos.Reference.Builder builder = FSProtos.Reference.newBuilder();
195 builder.setRange(isTopFileRegion(getFileRegion())?
196 FSProtos.Reference.Range.TOP: FSProtos.Reference.Range.BOTTOM);
197 builder.setSplitkey(ByteString.copyFrom(getSplitKey()));
198 return builder.build();
199 }
200
201 static Reference convert(final FSProtos.Reference r) {
202 Reference result = new Reference();
203 result.splitkey = r.getSplitkey().toByteArray();
204 result.region = r.getRange() == FSProtos.Reference.Range.TOP? Range.top: Range.bottom;
205 return result;
206 }
207
208
209
210
211
212
213
214
215 byte [] toByteArray() throws IOException {
216 return ProtobufUtil.prependPBMagic(convert().toByteArray());
217 }
218 }