-
Notifications
You must be signed in to change notification settings - Fork 3.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
HBASE-22890 Verify the file integrity in persistent IOEngine #633
Conversation
ois = new ObjectInputStream(fis); | ||
ois = new ObjectInputStream(new FileInputStream(persistencePath)); | ||
// there are two situations that can cause verification to fail | ||
if (!((FileIOEngine) ioEngine).verifyFileIntegrity(persistencePath, ois)) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ioEngine can directly cast to FileIOEngine? It's already guarded by !ioEngine.isPersistent().
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
My fault, it's should be PersistentIOEngine.
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
Show resolved
Hide resolved
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
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) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is some thing new we are doing now. Previously this delete step was not there if the files exists, and no persistentPath been configured.
In fact am wondering why we need to pass the persistentPath and algo here now? We call the verify method and there pass the persistentPath and can pass algo or so also.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I like this idea, codes look cleaner.
* @return the IOEngine | ||
* @throws IOException | ||
*/ | ||
private IOEngine getIOEngineFromName(String ioEngineName, long capacity) | ||
private IOEngine getIOEngineFromName() |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
See below comment also. Actually there is no need to have any change here now. We do not need to pass any extra args to FileIOE constructor now. No need to even change the order of call to getIOEngineFromName() above right. Better keep only the req change IMHO.
* @return true if verify successfully | ||
*/ | ||
@Override | ||
public boolean verifyFileIntegrity(String persistentPath, ObjectInputStream ois) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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.
return true; | ||
} else { | ||
// if the persistent file is not start with PB_MAGIC, it's an old version file | ||
oldVersion = true; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we still need this oldVersion state here in FileIOE?
} | ||
|
||
@Override | ||
public boolean isOldVersion() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I see now this is been used. So my point was like read the checksum in BucketCache only. And let IT decide whether to do the checksum verify or not based on PBMagic.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I got your point.
Hi, @anoopsjohn, I have modified. |
💔 -1 overall
This message was automatically generated. |
💔 -1 overall
This message was automatically generated. |
int readLen = ois.read(persistentChecksum); | ||
if (readLen != length || !((PersistentIOEngine) ioEngine).verifyFileIntegrity( | ||
persistentChecksum, algorithm)) { | ||
return; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We can add an INFO log here? Or we have the log saying the verify fail in FileIOE?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We have log if verify fail in FileIOE.
if (persistencePath != null) { | ||
retrieveFromFile(bucketSizes); | ||
} else { | ||
((PersistentIOEngine) ioEngine).deleteCacheDataFile(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If we delete data file here not, there wont be any functional diff I believe. We were not doing it before. Any reason to do so now? Or we do this delete in master branch now? Not remembering exactly.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do not delete data file does not affect the function, but it will occupy storage space.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Data file can be large, since it is out-of-date and can't be restore, i'm fine with the deletion. Being kept is also good to me, only a matter of overwrite again.
try { | ||
byte[] calculateChecksum = calculateChecksum(algorithm); | ||
if (!Bytes.equals(persistentChecksum, calculateChecksum)) { | ||
throw new IOException("The persistent checksum is " + Bytes.toString(persistentChecksum) + |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The response from this method is in 2 ways for IOE or mismatch of checksum. In caller place we check for boolean return value basically. Can we just log the issue here and return boolean in any case?
return false; | ||
} catch (NoSuchAlgorithmException nsae) { | ||
LOG.error("No such algorithm " + algorithm, nsae); | ||
throw new RuntimeException(nsae); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What will happen if we throw RTE here? The FileIOE create itself will get failed or even more? Retrieve from persistent cached fail need not be a reason for the FileIOE to be failed. But here we have configured this to be a persistent FileIOE. Later when we try persist during RS shutdown, we will get failure any way (?). Are u considering that here for this RTE? Am not really sure this throw RTE is correct here. Thoughts
return true; | ||
} catch (IOException ioex) { | ||
LOG.error("File verification failed because of ", ioex); | ||
reinit(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why this reinit only in this case? In case of checksum mismatch no such thing happening no?
+ " total space under " + filePath + ", not enough for requested " | ||
+ StringUtils.byteDesc(sizePerFile); | ||
+ " total space under " + filePath + ", not enough for requested " | ||
+ StringUtils.byteDesc(sizePerFile); | ||
LOG.warn(msg); | ||
} | ||
rafs[i].setLength(sizePerFile); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
setLength() can change the last modified time of the file, so the persistent checksum and calculate checksum are not equal and the unit test failed. But I don't know why my local test can pass...... Anyway, I will fix this.
int readLen = ois.read(persistentChecksum); | ||
if (readLen != length || !((PersistentIOEngine) ioEngine).verifyFileIntegrity( | ||
persistentChecksum, algorithm)) { | ||
LOG.warn("Can't restore from file because of verification failed."); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Separate the if branch? readLen != length
doesn't mean Can't restore from file because of verification failed
File file = new File(filePath); | ||
if (file.length() != sizePerFile) { | ||
rafs[i].setLength(sizePerFile); | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
why this change?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
setLength() method will change file's last modified time. So if don't change this, the wrong time will be used to calculate checksum.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please add a comment block //
to clarify the purpose.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ya some fat comments here would be nice. Got why u have this check now.. I can think of a case though. Say we have a file based cache with one file and size was 10 GB. Now the restart of the RS happening. The cache is persisted also. Before restart the size is been increased to 20 GB. There is no truncate and ideally the cache get rebuilt. Only thing is after the restart the cache capacity is increased. But now as per the code, the length is changed here and so the last modified time and which will fail the verify phase. Is it some thing to be considered? Dont want much complex handling for this. Might not be a common case for persisted cache. Max what happening is we not able to retrieve persisted cache. But welcoming thinking/suggestion.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If I understand you correctly, you mean change the bucket cache size before restart RS. Actually, in retrieveFromFile()
method will check the bucket cache size:
if (capacitySize != cacheCapacity) throw new IOException("Mismatched cache capacity:" + StringUtils.byteDesc(capacitySize) + ", expected: " + StringUtils.byteDesc(cacheCapacity));
So change the bucket cache size will no longer retrieve from file.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yaya that was my Q.. Oh great.. I did not read whole the code.. Then its perfect. Pls add enough comments around to know why this check and set length is so imp. Great.
* @throws IOException the IOException | ||
*/ | ||
private void reinit() throws IOException { | ||
LOG.info("Delete the cache data file and Reinitialize the FileIOEngine."); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reinitialize
lower case for R
?
sb.append(getFileSize(filePath)); | ||
sb.append(file.lastModified()); | ||
} else { | ||
throw new IOException("Cache file: " + filePath + " is not exists."); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In fact, after FileIOE inited, the file path must be exist anyway. I know's it's the complaining in code check, so can we just swallow this IOE in this method, instead of throw to upper caller?
💔 -1 overall
This message was automatically generated. |
Ping @anoopsjohn, when you have time. I think it cleaner than before. |
💔 -1 overall
This message was automatically generated. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1
Great work.. Thanks for the perseverance
@Reidddddd @anoopsjohn Thank you for your patience and advice! |
💔 -1 overall
This message was automatically generated. |
HBASE-22890