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