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