View Javadoc

1   /**
2    * Copyright The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one or more
5    * contributor license agreements. See the NOTICE file distributed with this
6    * work for additional information regarding copyright ownership. The ASF
7    * licenses this file to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance with the License.
9    * 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, WITHOUT
15   * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
16   * License for the specific language governing permissions and limitations
17   * under the License.
18   */
19  package org.apache.hadoop.hbase.io.hfile.bucket;
20  
21  import java.io.File;
22  import java.io.IOException;
23  import java.io.RandomAccessFile;
24  import java.nio.ByteBuffer;
25  import java.nio.channels.ClosedByInterruptException;
26  import java.nio.channels.ClosedChannelException;
27  import java.nio.channels.FileChannel;
28  import java.security.MessageDigest;
29  import java.security.NoSuchAlgorithmException;
30  import java.util.Arrays;
31  import java.util.concurrent.locks.ReentrantLock;
32  
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.hbase.classification.InterfaceAudience;
36  import org.apache.hadoop.hbase.util.Bytes;
37  import org.apache.hadoop.util.Shell;
38  import org.apache.hadoop.util.StringUtils;
39  
40  /**
41   * IO engine that stores data to a file on the local file system.
42   */
43  @InterfaceAudience.Private
44  public class FileIOEngine implements PersistentIOEngine {
45    private static final Log LOG = LogFactory.getLog(FileIOEngine.class);
46    public static final String FILE_DELIMITER = ",";
47    private static final DuFileCommand DU = new DuFileCommand(new String[] {"du", ""});
48  
49    private final String[] filePaths;
50    private final FileChannel[] fileChannels;
51    private final RandomAccessFile[] rafs;
52    private final ReentrantLock[] channelLocks;
53  
54    private final long sizePerFile;
55    private final long capacity;
56  
57    private FileReadAccessor readAccessor = new FileReadAccessor();
58    private FileWriteAccessor writeAccessor = new FileWriteAccessor();
59  
60    public FileIOEngine(long capacity, String... filePaths) throws IOException {
61      this.sizePerFile = capacity / filePaths.length;
62      this.capacity = this.sizePerFile * filePaths.length;
63      this.filePaths = filePaths;
64      this.fileChannels = new FileChannel[filePaths.length];
65      this.rafs = new RandomAccessFile[filePaths.length];
66      this.channelLocks = new ReentrantLock[filePaths.length];
67      for (int i = 0; i < filePaths.length; i++) {
68        String filePath = filePaths[i];
69        try {
70          rafs[i] = new RandomAccessFile(filePath, "rw");
71          long totalSpace = new File(filePath).getTotalSpace();
72          if (totalSpace < sizePerFile) {
73            // The next setting length will throw exception,logging this message
74            // is just used for the detail reason of exception,
75            String msg = "Only " + StringUtils.byteDesc(totalSpace)
76              + " total space under " + filePath + ", not enough for requested "
77              + StringUtils.byteDesc(sizePerFile);
78            LOG.warn(msg);
79          }
80          File file = new File(filePath);
81          // setLength() method will change file's last modified time. So if don't do
82          // this check, wrong time will be used when calculating checksum.
83          if (file.length() != sizePerFile) {
84            rafs[i].setLength(sizePerFile);
85          }
86          fileChannels[i] = rafs[i].getChannel();
87          channelLocks[i] = new ReentrantLock();
88          LOG.info("Allocating cache " + StringUtils.byteDesc(sizePerFile)
89            + ", on the path: " + filePath);
90        } catch (IOException fex) {
91          LOG.error("Failed allocating cache on " + filePath, fex);
92          shutdown();
93          throw fex;
94        }
95      }
96    }
97  
98    @Override
99    public boolean verifyFileIntegrity(byte[] persistentChecksum, String algorithm) {
100     byte[] calculateChecksum = calculateChecksum(algorithm);
101     if (!Bytes.equals(persistentChecksum, calculateChecksum)) {
102       LOG.error("Mismatch of checksum! The persistent checksum is " +
103         Bytes.toString(persistentChecksum) + ", but the calculate checksum is " +
104         Bytes.toString(calculateChecksum));
105       return false;
106     }
107     return true;
108   }
109 
110   @Override
111   public String toString() {
112     return "ioengine=" + this.getClass().getSimpleName() + ", paths="
113         + Arrays.asList(filePaths) + ", capacity=" + String.format("%,d", this.capacity);
114   }
115 
116   /**
117    * File IO engine is always able to support persistent storage for the cache
118    * @return true
119    */
120   @Override
121   public boolean isPersistent() {
122     return true;
123   }
124 
125   /**
126    * Transfers data from file to the given byte buffer
127    * @param dstBuffer the given byte buffer into which bytes are to be written
128    * @param offset The offset in the file where the first byte to be read
129    * @return number of bytes read
130    * @throws IOException
131    */
132   @Override
133   public int read(ByteBuffer dstBuffer, long offset) throws IOException {
134     if (dstBuffer.remaining() != 0) {
135       return accessFile(readAccessor, dstBuffer, offset);
136     }
137     return 0;
138   }
139 
140   void closeFileChannels() {
141     for (FileChannel fileChannel: fileChannels) {
142       try {
143         fileChannel.close();
144       } catch (IOException e) {
145         LOG.warn("Failed to close FileChannel", e);
146       }
147     }
148   }
149 
150   /**
151    * Transfers data from the given byte buffer to file
152    * @param srcBuffer the given byte buffer from which bytes are to be read
153    * @param offset The offset in the file where the first byte to be written
154    * @throws IOException
155    */
156   @Override
157   public void write(ByteBuffer srcBuffer, long offset) throws IOException {
158     if (!srcBuffer.hasRemaining()) {
159       return;
160     }
161     accessFile(writeAccessor, srcBuffer, offset);
162   }
163 
164   /**
165    * Sync the data to file after writing
166    * @throws IOException
167    */
168   @Override
169   public void sync() throws IOException {
170     for (int i = 0; i < fileChannels.length; i++) {
171       try {
172         if (fileChannels[i] != null) {
173           fileChannels[i].force(true);
174         }
175       } catch (IOException ie) {
176         LOG.warn("Failed syncing data to " + this.filePaths[i]);
177         throw ie;
178       }
179     }
180   }
181 
182   /**
183    * Close the file
184    */
185   @Override
186   public void shutdown() {
187     for (int i = 0; i < filePaths.length; i++) {
188       try {
189         if (fileChannels[i] != null) {
190           fileChannels[i].close();
191         }
192         if (rafs[i] != null) {
193           rafs[i].close();
194         }
195       } catch (IOException ex) {
196         LOG.error("Failed closing " + filePaths[i] + " when shudown the IOEngine", ex);
197       }
198     }
199   }
200 
201   private int accessFile(FileAccessor accessor, ByteBuffer buffer, long globalOffset)
202       throws IOException {
203     int startFileNum = getFileNum(globalOffset);
204     int remainingAccessDataLen = buffer.remaining();
205     int endFileNum = getFileNum(globalOffset + remainingAccessDataLen - 1);
206     int accessFileNum = startFileNum;
207     long accessOffset = getAbsoluteOffsetInFile(accessFileNum, globalOffset);
208     int bufLimit = buffer.limit();
209     while (true) {
210       FileChannel fileChannel = fileChannels[accessFileNum];
211       int accessLen = 0;
212       if (endFileNum > accessFileNum) {
213         // short the limit;
214         buffer.limit((int) (buffer.limit() - remainingAccessDataLen + sizePerFile - accessOffset));
215       }
216       try {
217         accessLen = accessor.access(fileChannel, buffer, accessOffset);
218       } catch (ClosedByInterruptException e) {
219         throw e;
220       } catch (ClosedChannelException e) {
221         refreshFileConnection(accessFileNum, e);
222         continue;
223       }
224       // recover the limit
225       buffer.limit(bufLimit);
226       if (accessLen < remainingAccessDataLen) {
227         remainingAccessDataLen -= accessLen;
228         accessFileNum++;
229         accessOffset = 0;
230       } else {
231         break;
232       }
233       if (accessFileNum >= fileChannels.length) {
234         throw new IOException("Required data len " + StringUtils.byteDesc(buffer.remaining())
235             + " exceed the engine's capacity " + StringUtils.byteDesc(capacity) + " where offset="
236             + globalOffset);
237       }
238     }
239     return bufLimit;
240   }
241 
242   /**
243    * Get the absolute offset in given file with the relative global offset.
244    * @param fileNum
245    * @param globalOffset
246    * @return the absolute offset
247    */
248   private long getAbsoluteOffsetInFile(int fileNum, long globalOffset) {
249     return globalOffset - fileNum * sizePerFile;
250   }
251 
252   private int getFileNum(long offset) {
253     if (offset < 0) {
254       throw new IllegalArgumentException("Unexpected offset " + offset);
255     }
256     int fileNum = (int) (offset / sizePerFile);
257     if (fileNum >= fileChannels.length) {
258       throw new RuntimeException("Not expected offset " + offset + " where capacity=" + capacity);
259     }
260     return fileNum;
261   }
262 
263   FileChannel[] getFileChannels() {
264     return fileChannels;
265   }
266 
267   void refreshFileConnection(int accessFileNum, IOException ioe) throws IOException {
268     ReentrantLock channelLock = channelLocks[accessFileNum];
269     channelLock.lock();
270     try {
271       FileChannel fileChannel = fileChannels[accessFileNum];
272       if (fileChannel != null) {
273         // Don't re-open a channel if we were waiting on another
274         // thread to re-open the channel and it is now open.
275         if (fileChannel.isOpen()) {
276           return;
277         }
278         fileChannel.close();
279       }
280       LOG.warn("Caught ClosedChannelException accessing BucketCache, reopening file: "
281           + filePaths[accessFileNum], ioe);
282       rafs[accessFileNum] = new RandomAccessFile(filePaths[accessFileNum], "rw");
283       fileChannels[accessFileNum] = rafs[accessFileNum].getChannel();
284     } finally{
285       channelLock.unlock();
286     }
287   }
288 
289   @Override
290   public byte[] calculateChecksum(String algorithm) {
291     if (filePaths == null) {
292       return null;
293     }
294     try {
295       StringBuilder sb = new StringBuilder();
296       for (String filePath : filePaths){
297         File file = new File(filePath);
298         sb.append(filePath);
299         sb.append(getFileSize(filePath));
300         sb.append(file.lastModified());
301       }
302       MessageDigest messageDigest = MessageDigest.getInstance(algorithm);
303       messageDigest.update(Bytes.toBytes(sb.toString()));
304       return messageDigest.digest();
305     } catch (IOException ioex) {
306       LOG.error("Calculating checksum failed.", ioex);
307       return null;
308     } catch (NoSuchAlgorithmException e) {
309       LOG.error("No such algorithm : " + algorithm + "!");
310       return null;
311     }
312   }
313 
314   /**
315    * Using Linux command du to get file's real size
316    * @param filePath the file
317    * @return file's real size
318    * @throws IOException something happened like file not exists
319    */
320   private static long getFileSize(String filePath) throws IOException {
321     DU.setExecCommand(filePath);
322     DU.execute();
323     return Long.parseLong(DU.getOutput().split("\t")[0]);
324   }
325 
326   private static class DuFileCommand extends Shell.ShellCommandExecutor {
327     private String[] execCommand;
328 
329     DuFileCommand(String[] execString) {
330       super(execString);
331       execCommand = execString;
332     }
333 
334     void setExecCommand(String filePath) {
335       this.execCommand[1] = filePath;
336     }
337 
338     @Override
339     public String[] getExecString() {
340       return this.execCommand;
341     }
342   }
343 
344   private static interface FileAccessor {
345     int access(FileChannel fileChannel, ByteBuffer byteBuffer, long accessOffset)
346         throws IOException;
347   }
348 
349   private static class FileReadAccessor implements FileAccessor {
350     @Override
351     public int access(FileChannel fileChannel, ByteBuffer byteBuffer, long accessOffset)
352         throws IOException {
353       return fileChannel.read(byteBuffer, accessOffset);
354     }
355   }
356 
357   private static class FileWriteAccessor implements FileAccessor {
358     @Override
359     public int access(FileChannel fileChannel, ByteBuffer byteBuffer, long accessOffset)
360         throws IOException {
361       return fileChannel.write(byteBuffer, accessOffset);
362     }
363   }
364 
365 }