View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  package org.apache.hadoop.hbase.regionserver.wal;
21  
22  import java.io.IOException;
23  import java.lang.reflect.Field;
24  import java.lang.reflect.InvocationTargetException;
25  import java.util.TreeMap;
26  
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.classification.InterfaceAudience;
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.hbase.HConstants;
35  import org.apache.hadoop.io.SequenceFile;
36  import org.apache.hadoop.io.Text;
37  import org.apache.hadoop.io.SequenceFile.CompressionType;
38  import org.apache.hadoop.io.SequenceFile.Metadata;
39  import org.apache.hadoop.io.compress.CompressionCodec;
40  import org.apache.hadoop.io.compress.DefaultCodec;
41  
42  /**
43   * Implementation of {@link HLog.Writer} that delegates to
44   * SequenceFile.Writer.
45   */
46  @InterfaceAudience.Private
47  public class SequenceFileLogWriter implements HLog.Writer {
48    static final Text WAL_VERSION_KEY = new Text("version");
49    // Let the version be 1.  Let absence of a version meta tag be old, version 0.
50    // Set this version '1' to be the version that introduces compression,
51    // the COMPRESSION_VERSION.
52    private static final int COMPRESSION_VERSION = 1;
53    static final int VERSION = COMPRESSION_VERSION;
54    static final Text WAL_VERSION = new Text("" + VERSION);
55    static final Text WAL_COMPRESSION_TYPE_KEY = new Text("compression.type");
56    static final Text DICTIONARY_COMPRESSION_TYPE = new Text("dictionary");
57  
58    private final Log LOG = LogFactory.getLog(this.getClass());
59    // The sequence file we delegate to.
60    private SequenceFile.Writer writer;
61    // This is the FSDataOutputStream instance that is the 'out' instance
62    // in the SequenceFile.Writer 'writer' instance above.
63    private FSDataOutputStream writer_out;
64  
65    private Class<? extends HLogKey> keyClass;
66  
67    /**
68     * Context used by our wal dictionary compressor.  Null if we're not to do
69     * our custom dictionary compression.  This custom WAL compression is distinct
70     * from sequencefile native compression.
71     */
72    private CompressionContext compressionContext;
73  
74    /**
75     * Default constructor.
76     */
77    public SequenceFileLogWriter() {
78      super();
79    }
80  
81    /**
82     * This constructor allows a specific HLogKey implementation to override that
83     * which would otherwise be chosen via configuration property.
84     * 
85     * @param keyClass
86     */
87    public SequenceFileLogWriter(Class<? extends HLogKey> keyClass) {
88      this.keyClass = keyClass;
89    }
90  
91    /**
92     * Create sequence file Metadata for our WAL file with version and compression
93     * type (if any).
94     * @param conf
95     * @param compress
96     * @return Metadata instance.
97     */
98    private static Metadata createMetadata(final Configuration conf,
99        final boolean compress) {
100     TreeMap<Text, Text> metaMap = new TreeMap<Text, Text>();
101     metaMap.put(WAL_VERSION_KEY, WAL_VERSION);
102     if (compress) {
103       // Currently we only do one compression type.
104       metaMap.put(WAL_COMPRESSION_TYPE_KEY, DICTIONARY_COMPRESSION_TYPE);
105     }
106     return new Metadata(metaMap);
107   }
108 
109   /**
110    * Call this method after init() has been executed
111    * 
112    * @return whether WAL compression is enabled
113    */
114   static boolean isWALCompressionEnabled(final Metadata metadata) {
115     // Check version is >= VERSION?
116     Text txt = metadata.get(WAL_VERSION_KEY);
117     if (txt == null || Integer.parseInt(txt.toString()) < COMPRESSION_VERSION) {
118       return false;
119     }
120     // Now check that compression type is present.  Currently only one value.
121     txt = metadata.get(WAL_COMPRESSION_TYPE_KEY);
122     return txt != null && txt.equals(DICTIONARY_COMPRESSION_TYPE);
123   }
124 
125   @Override
126   public void init(FileSystem fs, Path path, Configuration conf)
127   throws IOException {
128     // Should we do our custom WAL compression?
129     boolean compress = conf.getBoolean(HConstants.ENABLE_WAL_COMPRESSION, false);
130     if (compress) {
131       try {
132         if (this.compressionContext == null) {
133           this.compressionContext = new CompressionContext(LRUDictionary.class);
134         } else {
135           this.compressionContext.clear();
136         }
137       } catch (Exception e) {
138         throw new IOException("Failed to initiate CompressionContext", e);
139       }
140     }
141 
142     if (null == keyClass) {
143       keyClass = HLogUtil.getKeyClass(conf);
144     }
145 
146     // Create a SF.Writer instance.
147     try {
148       // reflection for a version of SequenceFile.createWriter that doesn't
149       // automatically create the parent directory (see HBASE-2312)
150       this.writer = (SequenceFile.Writer) SequenceFile.class
151         .getMethod("createWriter", new Class[] {FileSystem.class,
152             Configuration.class, Path.class, Class.class, Class.class,
153             Integer.TYPE, Short.TYPE, Long.TYPE, Boolean.TYPE,
154             CompressionType.class, CompressionCodec.class, Metadata.class})
155         .invoke(null, new Object[] {fs, conf, path, HLogUtil.getKeyClass(conf),
156             WALEdit.class,
157             Integer.valueOf(fs.getConf().getInt("io.file.buffer.size", 4096)),
158             Short.valueOf((short)
159               conf.getInt("hbase.regionserver.hlog.replication",
160               fs.getDefaultReplication())),
161             Long.valueOf(conf.getLong("hbase.regionserver.hlog.blocksize",
162                 fs.getDefaultBlockSize())),
163             Boolean.valueOf(false) /*createParent*/,
164             SequenceFile.CompressionType.NONE, new DefaultCodec(),
165             createMetadata(conf, compress)
166             });
167     } catch (InvocationTargetException ite) {
168       // function was properly called, but threw it's own exception
169       throw new IOException(ite.getCause());
170     } catch (Exception e) {
171       // ignore all other exceptions. related to reflection failure
172     }
173 
174     // if reflection failed, use the old createWriter
175     if (this.writer == null) {
176       LOG.debug("new createWriter -- HADOOP-6840 -- not available");
177       this.writer = SequenceFile.createWriter(fs, conf, path,
178         HLogUtil.getKeyClass(conf), WALEdit.class,
179         fs.getConf().getInt("io.file.buffer.size", 4096),
180         (short) conf.getInt("hbase.regionserver.hlog.replication",
181           fs.getDefaultReplication()),
182         conf.getLong("hbase.regionserver.hlog.blocksize",
183           fs.getDefaultBlockSize()),
184         SequenceFile.CompressionType.NONE,
185         new DefaultCodec(),
186         null,
187         createMetadata(conf, compress));
188     } else {
189       LOG.debug("using new createWriter -- HADOOP-6840");
190     }
191     
192     this.writer_out = getSequenceFilePrivateFSDataOutputStreamAccessible();
193     LOG.debug("Path=" + path + ", compression=" + compress);
194   }
195 
196   // Get at the private FSDataOutputStream inside in SequenceFile so we can
197   // call sync on it.  Make it accessible.
198   private FSDataOutputStream getSequenceFilePrivateFSDataOutputStreamAccessible()
199   throws IOException {
200     FSDataOutputStream out = null;
201     final Field fields [] = this.writer.getClass().getDeclaredFields();
202     final String fieldName = "out";
203     for (int i = 0; i < fields.length; ++i) {
204       if (fieldName.equals(fields[i].getName())) {
205         try {
206           // Make the 'out' field up in SF.Writer accessible.
207           fields[i].setAccessible(true);
208           out = (FSDataOutputStream)fields[i].get(this.writer);
209           break;
210         } catch (IllegalAccessException ex) {
211           throw new IOException("Accessing " + fieldName, ex);
212         } catch (SecurityException e) {
213           LOG.warn("Does not have access to out field from FSDataOutputStream",
214               e);
215         }
216       }
217     }
218     return out;
219   }
220 
221   @Override
222   public void append(HLog.Entry entry) throws IOException {
223     entry.setCompressionContext(compressionContext);
224     try {
225       this.writer.append(entry.getKey(), entry.getEdit());
226     } catch (NullPointerException npe) {
227       // Concurrent close...
228       throw new IOException(npe);
229     }
230   }
231 
232   @Override
233   public void close() throws IOException {
234     if (this.writer != null) {
235       try {
236         this.writer.close();
237       } catch (NullPointerException npe) {
238         // Can get a NPE coming up from down in DFSClient$DFSOutputStream#close
239         LOG.warn(npe);
240       }
241       this.writer = null;
242     }
243   }
244 
245   @Override
246   public void sync() throws IOException {
247     try {
248       this.writer.syncFs();
249     } catch (NullPointerException npe) {
250       // Concurrent close...
251       throw new IOException(npe);
252     }
253   }
254 
255   @Override
256   public long getLength() throws IOException {
257     try {
258       return this.writer.getLength();
259     } catch (NullPointerException npe) {
260       // Concurrent close...
261       throw new IOException(npe);
262     }
263   }
264 
265   /**
266    * @return The dfsclient out stream up inside SF.Writer made accessible, or
267    * null if not available.
268    */
269   public FSDataOutputStream getWriterFSDataOutputStream() {
270     return this.writer_out;
271   }
272 }