anoopsjohn commented on a change in pull request #633: HBASE-22890 Verify the 
file integrity in persistent IOEngine
URL: https://github.com/apache/hbase/pull/633#discussion_r325500780
 
 

 ##########
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
 ##########
 @@ -20,45 +20,107 @@
 
 import java.io.File;
 import java.io.IOException;
+import java.io.ObjectInputStream;
 import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.channels.ClosedByInterruptException;
 import java.nio.channels.ClosedChannelException;
 import java.nio.channels.FileChannel;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
 import java.util.Arrays;
 import java.util.concurrent.locks.ReentrantLock;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.StringUtils;
 
 /**
  * IO engine that stores data to a file on the local file system.
  */
 @InterfaceAudience.Private
-public class FileIOEngine implements IOEngine {
+public class FileIOEngine implements PersistentIOEngine {
   private static final Log LOG = LogFactory.getLog(FileIOEngine.class);
   public static final String FILE_DELIMITER = ",";
+  private static final DuFileCommand DU = new DuFileCommand(new String[] 
{"du", ""});
+
   private final String[] filePaths;
   private final FileChannel[] fileChannels;
   private final RandomAccessFile[] rafs;
   private final ReentrantLock[] channelLocks;
 
   private final long sizePerFile;
   private final long capacity;
+  private final String algorithmName;
+  /**
+   * Whether the persistent file support verify file integrity, old version 
file
+   * does not support verification,
+   */
+  private boolean oldVersion;
 
   private FileReadAccessor readAccessor = new FileReadAccessor();
   private FileWriteAccessor writeAccessor = new FileWriteAccessor();
 
-  public FileIOEngine(long capacity, String... filePaths) throws IOException {
+  public FileIOEngine(String algorithmName, String persistentPath,
+    long capacity, String... filePaths) throws IOException {
     this.sizePerFile = capacity / filePaths.length;
     this.capacity = this.sizePerFile * filePaths.length;
     this.filePaths = filePaths;
     this.fileChannels = new FileChannel[filePaths.length];
     this.rafs = new RandomAccessFile[filePaths.length];
     this.channelLocks = new ReentrantLock[filePaths.length];
+    this.algorithmName = algorithmName;
+    // not configure persistent path
+    if (persistentPath == null) {
+      deleteCacheDataFile();
+    }
+    init();
+  }
+
+  /**
+   * Verify cache files's integrity
+   * @param persistentPath the backingMap persistent path
+   * @param ois the ObjectInputStream to read persistent file
+   * @return true if verify successfully
+   */
+  @Override
+  public boolean verifyFileIntegrity(String persistentPath, ObjectInputStream 
ois)
 
 Review comment:
   Ah..  Can we pass the stored checksum only here? Actually the write of the 
checksum was done by BucketCache right?  So expecting the FileIOE to know that 
looks bit ugly. I mean its like PBMagic and then an int for checkusm length and 
so on.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to