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  import java.util.TreeMap;
29  
30  import org.apache.commons.logging.Log;
31  import org.apache.commons.logging.LogFactory;
32  import org.apache.hadoop.conf.Configuration;
33  import org.apache.hadoop.fs.FSDataOutputStream;
34  import org.apache.hadoop.fs.FileSystem;
35  import org.apache.hadoop.fs.Path;
36  import org.apache.hadoop.hbase.HConstants;
37  import org.apache.hadoop.io.SequenceFile;
38  import org.apache.hadoop.io.Text;
39  import org.apache.hadoop.io.SequenceFile.CompressionType;
40  import org.apache.hadoop.io.SequenceFile.Metadata;
41  import org.apache.hadoop.io.compress.CompressionCodec;
42  import org.apache.hadoop.io.compress.DefaultCodec;
43  
44  /**
45   * Implementation of {@link HLog.Writer} that delegates to
46   * SequenceFile.Writer.
47   */
48  public class SequenceFileLogWriter implements HLog.Writer {
49    static final Text WAL_VERSION_KEY = new Text("version");
50    // Let the version be 1.  Let absence of a version meta tag be old, version 0.
51    // Set this version '1' to be the version that introduces compression,
52    // the COMPRESSION_VERSION.
53    private static final int COMPRESSION_VERSION = 1;
54    static final int VERSION = COMPRESSION_VERSION;
55    static final Text WAL_VERSION = new Text("" + VERSION);
56    static final Text WAL_COMPRESSION_TYPE_KEY = new Text("compression.type");
57    static final Text DICTIONARY_COMPRESSION_TYPE = new Text("dictionary");
58  
59    private final Log LOG = LogFactory.getLog(this.getClass());
60    // The sequence file we delegate to.
61    private SequenceFile.Writer writer;
62    // This is the FSDataOutputStream instance that is the 'out' instance
63    // in the SequenceFile.Writer 'writer' instance above.
64    private FSDataOutputStream writer_out;
65  
66    private Class<? extends HLogKey> keyClass;
67  
68    /**
69     * Context used by our wal dictionary compressor.  Null if we're not to do
70     * our custom dictionary compression.  This custom WAL compression is distinct
71     * from sequencefile native compression.
72     */
73    private CompressionContext compressionContext;
74  
75    private Method syncFs = null;
76    private Method hflush = null;
77  
78    /**
79     * Default constructor.
80     */
81    public SequenceFileLogWriter() {
82      super();
83    }
84  
85    /**
86     * This constructor allows a specific HLogKey implementation to override that
87     * which would otherwise be chosen via configuration property.
88     * 
89     * @param keyClass
90     */
91    public SequenceFileLogWriter(Class<? extends HLogKey> keyClass) {
92      this.keyClass = keyClass;
93    }
94  
95    /**
96     * Create sequence file Metadata for our WAL file with version and compression
97     * type (if any).
98     * @param conf
99     * @param compress
100    * @return Metadata instance.
101    */
102   private static Metadata createMetadata(final Configuration conf,
103       final boolean compress) {
104     TreeMap<Text, Text> metaMap = new TreeMap<Text, Text>();
105     metaMap.put(WAL_VERSION_KEY, WAL_VERSION);
106     if (compress) {
107       // Currently we only do one compression type.
108       metaMap.put(WAL_COMPRESSION_TYPE_KEY, DICTIONARY_COMPRESSION_TYPE);
109     }
110     return new Metadata(metaMap);
111   }
112 
113   /**
114    * Call this method after init() has been executed
115    * 
116    * @return whether WAL compression is enabled
117    */
118   static boolean isWALCompressionEnabled(final Metadata metadata) {
119     // Check version is >= VERSION?
120     Text txt = metadata.get(WAL_VERSION_KEY);
121     if (txt == null || Integer.parseInt(txt.toString()) < COMPRESSION_VERSION) {
122       return false;
123     }
124     // Now check that compression type is present.  Currently only one value.
125     txt = metadata.get(WAL_COMPRESSION_TYPE_KEY);
126     return txt != null && txt.equals(DICTIONARY_COMPRESSION_TYPE);
127   }
128 
129   @Override
130   public void init(FileSystem fs, Path path, Configuration conf)
131   throws IOException {
132     // Should we do our custom WAL compression?
133     boolean compress = conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
134     if (compress) {
135       try {
136         if (this.compressionContext == null) {
137           this.compressionContext = new CompressionContext(LRUDictionary.class);
138         } else {
139           this.compressionContext.clear();
140         }
141       } catch (Exception e) {
142         throw new IOException("Failed to initiate CompressionContext", e);
143       }
144     }
145 
146     if (null == keyClass) {
147       keyClass = HLog.getKeyClass(conf);
148     }
149 
150     // Create a SF.Writer instance.
151     try {
152       // reflection for a version of SequenceFile.createWriter that doesn't
153       // automatically create the parent directory (see HBASE-2312)
154       this.writer = (SequenceFile.Writer) SequenceFile.class
155         .getMethod("createWriter", new Class[] {FileSystem.class,
156             Configuration.class, Path.class, Class.class, Class.class,
157             Integer.TYPE, Short.TYPE, Long.TYPE, Boolean.TYPE,
158             CompressionType.class, CompressionCodec.class, Metadata.class})
159         .invoke(null, new Object[] {fs, conf, path, HLog.getKeyClass(conf),
160             WALEdit.class,
161             Integer.valueOf(fs.getConf().getInt("io.file.buffer.size", 4096)),
162             Short.valueOf((short)
163               conf.getInt("hbase.regionserver.hlog.replication",
164               fs.getDefaultReplication())),
165             Long.valueOf(conf.getLong("hbase.regionserver.hlog.blocksize",
166                 fs.getDefaultBlockSize())),
167             Boolean.valueOf(false) /*createParent*/,
168             SequenceFile.CompressionType.NONE, new DefaultCodec(),
169             createMetadata(conf, compress)
170             });
171     } catch (InvocationTargetException ite) {
172       // function was properly called, but threw it's own exception
173       throw new IOException(ite.getCause());
174     } catch (Exception e) {
175       // ignore all other exceptions. related to reflection failure
176     }
177 
178     // if reflection failed, use the old createWriter
179     if (this.writer == null) {
180       LOG.debug("new createWriter -- HADOOP-6840 -- not available");
181       this.writer = SequenceFile.createWriter(fs, conf, path,
182         HLog.getKeyClass(conf), WALEdit.class,
183         fs.getConf().getInt("io.file.buffer.size", 4096),
184         (short) conf.getInt("hbase.regionserver.hlog.replication",
185           fs.getDefaultReplication()),
186         conf.getLong("hbase.regionserver.hlog.blocksize",
187           fs.getDefaultBlockSize()),
188         SequenceFile.CompressionType.NONE,
189         new DefaultCodec(),
190         null,
191         createMetadata(conf, compress));
192     } else {
193       LOG.debug("using new createWriter -- HADOOP-6840");
194     }
195     
196     this.writer_out = getSequenceFilePrivateFSDataOutputStreamAccessible();
197     this.syncFs = getSyncFs();
198     this.hflush = getHFlush();
199     String msg = "Path=" + path +
200       ", syncFs=" + (this.syncFs != null) +
201       ", hflush=" + (this.hflush != null) +
202       ", compression=" + compress;
203     if (this.syncFs != null || this.hflush != null) {
204       LOG.debug(msg);
205     } else {
206       LOG.warn("No sync support! " + msg);
207     }
208   }
209 
210   /**
211    * Now do dirty work to see if syncFs is available on the backing this.writer.
212    * It will be available in branch-0.20-append and in CDH3.
213    * @return The syncFs method or null if not available.
214    * @throws IOException
215    */
216   private Method getSyncFs()
217   throws IOException {
218     Method m = null;
219     try {
220       // function pointer to writer.syncFs() method; present when sync is hdfs-200.
221       m = this.writer.getClass().getMethod("syncFs", new Class<?> []{});
222     } catch (SecurityException e) {
223       throw new IOException("Failed test for syncfs", e);
224     } catch (NoSuchMethodException e) {
225       // Not available
226     }
227     return m;
228   }
229 
230   /**
231    * See if hflush (0.21 and 0.22 hadoop) is available.
232    * @return The hflush method or null if not available.
233    * @throws IOException
234    */
235   private Method getHFlush()
236   throws IOException {
237     Method m = null;
238     try {
239       Class<? extends OutputStream> c = getWriterFSDataOutputStream().getClass();
240       m = c.getMethod("hflush", new Class<?> []{});
241     } catch (SecurityException e) {
242       throw new IOException("Failed test for hflush", e);
243     } catch (NoSuchMethodException e) {
244       // Ignore
245     }
246     return m;
247   }
248 
249   // Get at the private FSDataOutputStream inside in SequenceFile so we can
250   // call sync on it.  Make it accessible.
251   private FSDataOutputStream getSequenceFilePrivateFSDataOutputStreamAccessible()
252   throws IOException {
253     FSDataOutputStream out = null;
254     final Field fields [] = this.writer.getClass().getDeclaredFields();
255     final String fieldName = "out";
256     for (int i = 0; i < fields.length; ++i) {
257       if (fieldName.equals(fields[i].getName())) {
258         try {
259           // Make the 'out' field up in SF.Writer accessible.
260           fields[i].setAccessible(true);
261           out = (FSDataOutputStream)fields[i].get(this.writer);
262           break;
263         } catch (IllegalAccessException ex) {
264           throw new IOException("Accessing " + fieldName, ex);
265         } catch (SecurityException e) {
266           // TODO Auto-generated catch block
267           e.printStackTrace();
268         }
269       }
270     }
271     return out;
272   }
273 
274   @Override
275   public void append(HLog.Entry entry) throws IOException {
276     entry.setCompressionContext(compressionContext);
277     try {
278       this.writer.append(entry.getKey(), entry.getEdit());
279     } catch (NullPointerException npe) {
280       // Concurrent close...
281       throw new IOException(npe);
282     }
283   }
284 
285   @Override
286   public void close() throws IOException {
287     if (this.writer != null) {
288       try {
289         this.writer.close();
290       } catch (NullPointerException npe) {
291         // Can get a NPE coming up from down in DFSClient$DFSOutputStream#close
292         LOG.warn(npe);
293       }
294       this.writer = null;
295     }
296   }
297 
298   @Override
299   public void sync() throws IOException {
300     if (this.syncFs != null) {
301       try {
302        this.syncFs.invoke(this.writer, HLog.NO_ARGS);
303       } catch (Exception e) {
304         throw new IOException("Reflection", e);
305       }
306     } else if (this.hflush != null) {
307       try {
308         this.hflush.invoke(getWriterFSDataOutputStream(), HLog.NO_ARGS);
309       } catch (Exception e) {
310         throw new IOException("Reflection", e);
311       }
312     }
313   }
314 
315   @Override
316   public long getLength() throws IOException {
317     try {
318       return this.writer.getLength();
319     } catch (NullPointerException npe) {
320       // Concurrent close...
321       throw new IOException(npe);
322     }
323   }
324 
325   /**
326    * @return The dfsclient out stream up inside SF.Writer made accessible, or
327    * null if not available.
328    */
329   public FSDataOutputStream getWriterFSDataOutputStream() {
330     return this.writer_out;
331   }
332 }