1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21 package org.apache.hadoop.hbase.io.hfile;
22
23 import java.io.DataOutput;
24 import java.io.IOException;
25 import java.util.ArrayList;
26 import java.util.Arrays;
27 import java.util.List;
28
29 import org.apache.hadoop.conf.Configuration;
30 import org.apache.hadoop.fs.FSDataOutputStream;
31 import org.apache.hadoop.fs.FileSystem;
32 import org.apache.hadoop.fs.Path;
33 import org.apache.hadoop.fs.permission.FsPermission;
34 import org.apache.hadoop.hbase.KeyValue.KeyComparator;
35 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
36 import org.apache.hadoop.hbase.util.Bytes;
37 import org.apache.hadoop.io.RawComparator;
38 import org.apache.hadoop.io.Writable;
39
40
41
42
43 public abstract class AbstractHFileWriter implements HFile.Writer {
44
45
46 protected byte[] lastKeyBuffer = null;
47
48 protected int lastKeyOffset = -1;
49 protected int lastKeyLength = -1;
50
51
52 protected FSDataOutputStream outputStream;
53
54
55 protected final boolean closeOutputStream;
56
57
58 protected FileInfo fileInfo = new HFile.FileInfo();
59
60
61 protected final int blockSize;
62
63
64 protected long entryCount = 0;
65
66
67 protected long totalKeyLength = 0;
68
69
70 protected long totalValueLength = 0;
71
72
73 protected long totalUncompressedBytes = 0;
74
75
76 protected final RawComparator<byte[]> comparator;
77
78
79 protected List<byte[]> metaNames = new ArrayList<byte[]>();
80
81
82 protected List<Writable> metaData = new ArrayList<Writable>();
83
84
85 protected final Compression.Algorithm compressAlgo;
86
87
88 protected byte[] firstKeyInBlock = null;
89
90
91 protected final Path path;
92
93
94 protected final CacheConfig cacheConf;
95
96
97
98
99
100 protected final String name;
101
102 public AbstractHFileWriter(CacheConfig cacheConf,
103 FSDataOutputStream outputStream, Path path, int blockSize,
104 Compression.Algorithm compressAlgo, KeyComparator comparator) {
105 this.outputStream = outputStream;
106 this.path = path;
107 this.name = path != null ? path.getName() : outputStream.toString();
108 this.blockSize = blockSize;
109 this.compressAlgo = compressAlgo == null
110 ? HFile.DEFAULT_COMPRESSION_ALGORITHM : compressAlgo;
111 this.comparator = comparator != null ? comparator
112 : Bytes.BYTES_RAWCOMPARATOR;
113
114 closeOutputStream = path != null;
115 this.cacheConf = cacheConf;
116 }
117
118
119
120
121 protected void finishFileInfo() throws IOException {
122 if (lastKeyBuffer != null) {
123
124
125 fileInfo.append(FileInfo.LASTKEY, Arrays.copyOfRange(lastKeyBuffer,
126 lastKeyOffset, lastKeyOffset + lastKeyLength), false);
127 }
128
129
130 int avgKeyLen =
131 entryCount == 0 ? 0 : (int) (totalKeyLength / entryCount);
132 fileInfo.append(FileInfo.AVG_KEY_LEN, Bytes.toBytes(avgKeyLen), false);
133
134
135 int avgValueLen =
136 entryCount == 0 ? 0 : (int) (totalValueLength / entryCount);
137 fileInfo.append(FileInfo.AVG_VALUE_LEN, Bytes.toBytes(avgValueLen), false);
138 }
139
140
141
142
143
144
145
146
147
148 @Override
149 public void appendFileInfo(final byte[] k, final byte[] v)
150 throws IOException {
151 fileInfo.append(k, v, true);
152 }
153
154
155
156
157
158
159
160
161
162
163
164 protected final void writeFileInfo(FixedFileTrailer trailer, DataOutput out)
165 throws IOException {
166 trailer.setFileInfoOffset(outputStream.getPos());
167 finishFileInfo();
168 fileInfo.write(out);
169 }
170
171
172
173
174
175
176
177
178 protected boolean checkKey(final byte[] key, final int offset,
179 final int length) throws IOException {
180 boolean isDuplicateKey = false;
181
182 if (key == null || length <= 0) {
183 throw new IOException("Key cannot be null or empty");
184 }
185 if (length > HFile.MAXIMUM_KEY_LENGTH) {
186 throw new IOException("Key length " + length + " > "
187 + HFile.MAXIMUM_KEY_LENGTH);
188 }
189 if (lastKeyBuffer != null) {
190 int keyComp = comparator.compare(lastKeyBuffer, lastKeyOffset,
191 lastKeyLength, key, offset, length);
192 if (keyComp > 0) {
193 throw new IOException("Added a key not lexically larger than"
194 + " previous key="
195 + Bytes.toStringBinary(key, offset, length)
196 + ", lastkey="
197 + Bytes.toStringBinary(lastKeyBuffer, lastKeyOffset,
198 lastKeyLength));
199 } else if (keyComp == 0) {
200 isDuplicateKey = true;
201 }
202 }
203 return isDuplicateKey;
204 }
205
206
207 protected void checkValue(final byte[] value, final int offset,
208 final int length) throws IOException {
209 if (value == null) {
210 throw new IOException("Value cannot be null");
211 }
212 }
213
214
215
216
217 @Override
218 public Path getPath() {
219 return path;
220 }
221
222 @Override
223 public String toString() {
224 return "writer=" + (path != null ? path.toString() : null) + ", name="
225 + name + ", compression=" + compressAlgo.getName();
226 }
227
228
229
230
231
232 protected void finishClose(FixedFileTrailer trailer) throws IOException {
233 trailer.setMetaIndexCount(metaNames.size());
234 trailer.setTotalUncompressedBytes(totalUncompressedBytes+ trailer.getTrailerSize());
235 trailer.setEntryCount(entryCount);
236 trailer.setCompressionCodec(compressAlgo);
237
238 trailer.serialize(outputStream);
239
240 if (closeOutputStream) {
241 outputStream.close();
242 outputStream = null;
243 }
244 }
245
246 public static Compression.Algorithm compressionByName(String algoName) {
247 if (algoName == null)
248 return HFile.DEFAULT_COMPRESSION_ALGORITHM;
249 return Compression.getCompressionAlgorithmByName(algoName);
250 }
251
252
253 protected static FSDataOutputStream createOutputStream(Configuration conf,
254 FileSystem fs, Path path) throws IOException {
255 return fs.create(path, FsPermission.getDefault(), true,
256 fs.getConf().getInt("io.file.buffer.size", 4096),
257 fs.getDefaultReplication(), fs.getDefaultBlockSize(),
258 null);
259 }
260 }