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