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