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.Method;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.fs.FSDataOutputStream;
32 import org.apache.hadoop.fs.FileSystem;
33 import org.apache.hadoop.fs.Path;
34 import org.apache.hadoop.io.SequenceFile;
35 import org.apache.hadoop.io.SequenceFile.Metadata;
36 import org.apache.hadoop.io.compress.DefaultCodec;
37
38
39
40
41
42 public class SequenceFileLogWriter implements HLog.Writer {
43 private final Log LOG = LogFactory.getLog(this.getClass());
44
45 private SequenceFile.Writer writer;
46
47 private OutputStream dfsClient_out;
48
49 private Method syncFs;
50
51 public SequenceFileLogWriter() {
52 super();
53 }
54
55 @Override
56 public void init(FileSystem fs, Path path, Configuration conf)
57 throws IOException {
58
59 this.writer = SequenceFile.createWriter(fs, conf, path,
60 HLog.getKeyClass(conf), WALEdit.class,
61 fs.getConf().getInt("io.file.buffer.size", 4096),
62 (short) conf.getInt("hbase.regionserver.hlog.replication",
63 fs.getDefaultReplication()),
64 conf.getLong("hbase.regionserver.hlog.blocksize",
65 fs.getDefaultBlockSize()),
66 SequenceFile.CompressionType.NONE,
67 new DefaultCodec(),
68 null,
69 new Metadata());
70
71
72
73
74 final Field fields [] = this.writer.getClass().getDeclaredFields();
75 final String fieldName = "out";
76 for (int i = 0; i < fields.length; ++i) {
77 if (fieldName.equals(fields[i].getName())) {
78 try {
79
80 fields[i].setAccessible(true);
81 FSDataOutputStream out =
82 (FSDataOutputStream)fields[i].get(this.writer);
83 this.dfsClient_out = out.getWrappedStream();
84 break;
85 } catch (IllegalAccessException ex) {
86 throw new IOException("Accessing " + fieldName, ex);
87 }
88 }
89 }
90
91
92
93 Method m = null;
94 if (conf.getBoolean("dfs.support.append", false)) {
95 try {
96
97 m = this.writer.getClass().getMethod("syncFs", new Class<?> []{});
98 } catch (SecurityException e) {
99 throw new IOException("Failed test for syncfs", e);
100 } catch (NoSuchMethodException e) {
101
102 }
103 }
104 this.syncFs = m;
105 LOG.info((this.syncFs != null)?
106 "Using syncFs -- HDFS-200": "syncFs -- HDFS-200 -- not available");
107 }
108
109 @Override
110 public void append(HLog.Entry entry) throws IOException {
111 this.writer.append(entry.getKey(), entry.getEdit());
112 }
113
114 @Override
115 public void close() throws IOException {
116 this.writer.close();
117 }
118
119 @Override
120 public void sync() throws IOException {
121 if (this.syncFs != null) {
122 try {
123 this.syncFs.invoke(this.writer, HLog.NO_ARGS);
124 } catch (Exception e) {
125 throw new IOException("Reflection", e);
126 }
127 }
128 }
129
130 @Override
131 public long getLength() throws IOException {
132 return this.writer.getLength();
133 }
134
135
136
137
138
139 public OutputStream getDFSCOutputStream() {
140 return this.dfsClient_out;
141 }
142 }