View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
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   * Implementation of {@link HLog.Writer} that delegates to
43   * SequenceFile.Writer.
44   */
45  public class SequenceFileLogWriter implements HLog.Writer {
46    private final Log LOG = LogFactory.getLog(this.getClass());
47    // The sequence file we delegate to.
48    private SequenceFile.Writer writer;
49    // This is the FSDataOutputStream instance that is the 'out' instance
50    // in the SequenceFile.Writer 'writer' instance above.
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     * Default constructor.
60     */
61    public SequenceFileLogWriter() {
62      super();
63    }
64  
65    /**
66     * This constructor allows a specific HLogKey implementation to override that
67     * which would otherwise be chosen via configuration property.
68     * 
69     * @param keyClass
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      // Create a SF.Writer instance.
84      try {
85        // reflection for a version of SequenceFile.createWriter that doesn't
86        // automatically create the parent directory (see HBASE-2312)
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) /*createParent*/,
101             SequenceFile.CompressionType.NONE, new DefaultCodec(),
102             new Metadata()
103             });
104     } catch (InvocationTargetException ite) {
105       // function was properly called, but threw it's own exception
106       throw new IOException(ite.getCause());
107     } catch (Exception e) {
108       // ignore all other exceptions. related to reflection failure
109     }
110 
111     // if reflection failed, use the old createWriter
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    * Now do dirty work to see if syncFs is available on the backing this.writer.
144    * It will be available in branch-0.20-append and in CDH3.
145    * @return The syncFs method or null if not available.
146    * @throws IOException
147    */
148   private Method getSyncFs()
149   throws IOException {
150     Method m = null;
151     try {
152       // function pointer to writer.syncFs() method; present when sync is hdfs-200.
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       // Not available
158     }
159     return m;
160   }
161 
162   /**
163    * See if hflush (0.21 and 0.22 hadoop) is available.
164    * @return The hflush method or null if not available.
165    * @throws IOException
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       // Ignore
177     }
178     return m;
179   }
180 
181   // Get at the private FSDataOutputStream inside in SequenceFile so we can
182   // call sync on it.  Make it accessible.
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           // Make the 'out' field up in SF.Writer accessible.
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           // TODO Auto-generated catch block
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    * @return The dfsclient out stream up inside SF.Writer made accessible, or
243    * null if not available.
244    */
245   public FSDataOutputStream getWriterFSDataOutputStream() {
246     return this.writer_out;
247   }
248 }