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