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.DataOutput;
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.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.HConstants;
33 import org.apache.hadoop.hbase.KeyValue.KeyComparator;
34 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
35 import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured;
36 import org.apache.hadoop.hbase.util.Bytes;
37 import org.apache.hadoop.hbase.util.FSUtils;
38 import org.apache.hadoop.io.RawComparator;
39 import org.apache.hadoop.io.Writable;
40
41
42
43
44 public abstract class AbstractHFileWriter extends SchemaConfigured
45 implements HFile.Writer {
46
47
48 protected byte[] lastKeyBuffer = null;
49
50 protected int lastKeyOffset = -1;
51 protected int lastKeyLength = -1;
52
53
54 protected FSDataOutputStream outputStream;
55
56
57 protected final boolean closeOutputStream;
58
59
60 protected FileInfo fileInfo = new HFile.FileInfo();
61
62
63 protected final int blockSize;
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 RawComparator<byte[]> 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 final Compression.Algorithm compressAlgo;
88
89
90
91
92
93 protected final HFileDataBlockEncoder blockEncoder;
94
95
96 protected byte[] firstKeyInBlock = null;
97
98
99 protected final Path path;
100
101
102
103 protected final CacheConfig cacheConf;
104
105
106
107
108
109 protected final String name;
110
111 public AbstractHFileWriter(CacheConfig cacheConf,
112 FSDataOutputStream outputStream, Path path, int blockSize,
113 Compression.Algorithm compressAlgo,
114 HFileDataBlockEncoder dataBlockEncoder,
115 KeyComparator comparator) {
116 super(null, path);
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, DataOutput 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 (length > HFile.MAXIMUM_KEY_LENGTH) {
200 throw new IOException("Key length " + length + " > "
201 + HFile.MAXIMUM_KEY_LENGTH);
202 }
203 if (lastKeyBuffer != null) {
204 int keyComp = comparator.compare(lastKeyBuffer, lastKeyOffset,
205 lastKeyLength, key, offset, length);
206 if (keyComp > 0) {
207 throw new IOException("Added a key not lexically larger than"
208 + " previous key="
209 + Bytes.toStringBinary(key, offset, length)
210 + ", lastkey="
211 + Bytes.toStringBinary(lastKeyBuffer, lastKeyOffset,
212 lastKeyLength));
213 } else if (keyComp == 0) {
214 isDuplicateKey = true;
215 }
216 }
217 return isDuplicateKey;
218 }
219
220
221 protected void checkValue(final byte[] value, final int offset,
222 final int length) throws IOException {
223 if (value == null) {
224 throw new IOException("Value cannot be null");
225 }
226 }
227
228
229
230
231 @Override
232 public Path getPath() {
233 return path;
234 }
235
236 @Override
237 public String toString() {
238 return "writer=" + (path != null ? path.toString() : null) + ", name="
239 + name + ", compression=" + compressAlgo.getName();
240 }
241
242
243
244
245
246 protected void finishClose(FixedFileTrailer trailer) throws IOException {
247 trailer.setMetaIndexCount(metaNames.size());
248 trailer.setTotalUncompressedBytes(totalUncompressedBytes+ trailer.getTrailerSize());
249 trailer.setEntryCount(entryCount);
250 trailer.setCompressionCodec(compressAlgo);
251
252 trailer.serialize(outputStream);
253
254 if (closeOutputStream) {
255 outputStream.close();
256 outputStream = null;
257 }
258 }
259
260 public static Compression.Algorithm compressionByName(String algoName) {
261 if (algoName == null)
262 return HFile.DEFAULT_COMPRESSION_ALGORITHM;
263 return Compression.getCompressionAlgorithmByName(algoName);
264 }
265
266
267 protected static FSDataOutputStream createOutputStream(Configuration conf,
268 FileSystem fs, Path path) throws IOException {
269 FsPermission perms = FSUtils.getFilePermissions(fs, conf,
270 HConstants.DATA_FILE_UMASK_KEY);
271 return FSUtils.create(fs, path, perms);
272 }
273 }