1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21 package org.apache.hadoop.hbase.regionserver.wal;
22
23 import java.io.IOException;
24 import java.io.OutputStream;
25 import java.lang.reflect.Field;
26 import java.lang.reflect.InvocationTargetException;
27 import java.lang.reflect.Method;
28
29 import org.apache.commons.logging.Log;
30 import org.apache.commons.logging.LogFactory;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.fs.FSDataOutputStream;
33 import org.apache.hadoop.fs.FileSystem;
34 import org.apache.hadoop.fs.Path;
35 import org.apache.hadoop.io.SequenceFile;
36 import org.apache.hadoop.io.SequenceFile.CompressionType;
37 import org.apache.hadoop.io.SequenceFile.Metadata;
38 import org.apache.hadoop.io.compress.CompressionCodec;
39 import org.apache.hadoop.io.compress.DefaultCodec;
40
41
42
43
44
45 public class SequenceFileLogWriter implements HLog.Writer {
46 private final Log LOG = LogFactory.getLog(this.getClass());
47
48 private SequenceFile.Writer writer;
49
50
51 private FSDataOutputStream writer_out;
52
53 private Class<? extends HLogKey> keyClass;
54
55 private Method syncFs = null;
56 private Method hflush = null;
57
58
59
60
61 public SequenceFileLogWriter() {
62 super();
63 }
64
65
66
67
68
69
70
71 public SequenceFileLogWriter(Class<? extends HLogKey> keyClass) {
72 this.keyClass = keyClass;
73 }
74
75 @Override
76 public void init(FileSystem fs, Path path, Configuration conf)
77 throws IOException {
78
79 if (null == keyClass) {
80 keyClass = HLog.getKeyClass(conf);
81 }
82
83
84 try {
85
86
87 this.writer = (SequenceFile.Writer) SequenceFile.class
88 .getMethod("createWriter", new Class[] {FileSystem.class,
89 Configuration.class, Path.class, Class.class, Class.class,
90 Integer.TYPE, Short.TYPE, Long.TYPE, Boolean.TYPE,
91 CompressionType.class, CompressionCodec.class, Metadata.class})
92 .invoke(null, new Object[] {fs, conf, path, HLog.getKeyClass(conf),
93 WALEdit.class,
94 new Integer(fs.getConf().getInt("io.file.buffer.size", 4096)),
95 new Short((short)
96 conf.getInt("hbase.regionserver.hlog.replication",
97 fs.getDefaultReplication())),
98 new Long(conf.getLong("hbase.regionserver.hlog.blocksize",
99 fs.getDefaultBlockSize())),
100 new Boolean(false)
101 SequenceFile.CompressionType.NONE, new DefaultCodec(),
102 new Metadata()
103 });
104 } catch (InvocationTargetException ite) {
105
106 throw new IOException(ite.getCause());
107 } catch (Exception e) {
108
109 }
110
111
112 if (this.writer == null) {
113 LOG.debug("new createWriter -- HADOOP-6840 -- not available");
114 this.writer = SequenceFile.createWriter(fs, conf, path,
115 HLog.getKeyClass(conf), WALEdit.class,
116 fs.getConf().getInt("io.file.buffer.size", 4096),
117 (short) conf.getInt("hbase.regionserver.hlog.replication",
118 fs.getDefaultReplication()),
119 conf.getLong("hbase.regionserver.hlog.blocksize",
120 fs.getDefaultBlockSize()),
121 SequenceFile.CompressionType.NONE,
122 new DefaultCodec(),
123 null,
124 new Metadata());
125 } else {
126 LOG.debug("using new createWriter -- HADOOP-6840");
127 }
128
129 this.writer_out = getSequenceFilePrivateFSDataOutputStreamAccessible();
130 this.syncFs = getSyncFs();
131 this.hflush = getHFlush();
132 String msg = "Path=" + path +
133 ", syncFs=" + (this.syncFs != null) +
134 ", hflush=" + (this.hflush != null);
135 if (this.syncFs != null || this.hflush != null) {
136 LOG.debug(msg);
137 } else {
138 LOG.warn("No sync support! " + msg);
139 }
140 }
141
142
143
144
145
146
147
148 private Method getSyncFs()
149 throws IOException {
150 Method m = null;
151 try {
152
153 m = this.writer.getClass().getMethod("syncFs", new Class<?> []{});
154 } catch (SecurityException e) {
155 throw new IOException("Failed test for syncfs", e);
156 } catch (NoSuchMethodException e) {
157
158 }
159 return m;
160 }
161
162
163
164
165
166
167 private Method getHFlush()
168 throws IOException {
169 Method m = null;
170 try {
171 Class<? extends OutputStream> c = getWriterFSDataOutputStream().getClass();
172 m = c.getMethod("hflush", new Class<?> []{});
173 } catch (SecurityException e) {
174 throw new IOException("Failed test for hflush", e);
175 } catch (NoSuchMethodException e) {
176
177 }
178 return m;
179 }
180
181
182
183 private FSDataOutputStream getSequenceFilePrivateFSDataOutputStreamAccessible()
184 throws IOException {
185 FSDataOutputStream out = null;
186 final Field fields [] = this.writer.getClass().getDeclaredFields();
187 final String fieldName = "out";
188 for (int i = 0; i < fields.length; ++i) {
189 if (fieldName.equals(fields[i].getName())) {
190 try {
191
192 fields[i].setAccessible(true);
193 out = (FSDataOutputStream)fields[i].get(this.writer);
194 break;
195 } catch (IllegalAccessException ex) {
196 throw new IOException("Accessing " + fieldName, ex);
197 } catch (SecurityException e) {
198
199 e.printStackTrace();
200 }
201 }
202 }
203 return out;
204 }
205
206 @Override
207 public void append(HLog.Entry entry) throws IOException {
208 this.writer.append(entry.getKey(), entry.getEdit());
209 }
210
211 @Override
212 public void close() throws IOException {
213 if (this.writer != null) {
214 this.writer.close();
215 this.writer = null;
216 }
217 }
218
219 @Override
220 public void sync() throws IOException {
221 if (this.syncFs != null) {
222 try {
223 this.syncFs.invoke(this.writer, HLog.NO_ARGS);
224 } catch (Exception e) {
225 throw new IOException("Reflection", e);
226 }
227 } else if (this.hflush != null) {
228 try {
229 this.hflush.invoke(getWriterFSDataOutputStream(), HLog.NO_ARGS);
230 } catch (Exception e) {
231 throw new IOException("Reflection", e);
232 }
233 }
234 }
235
236 @Override
237 public long getLength() throws IOException {
238 return this.writer.getLength();
239 }
240
241
242
243
244
245 public FSDataOutputStream getWriterFSDataOutputStream() {
246 return this.writer_out;
247 }
248 }