Skip to content
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-23584 : Descrease rpc getFileStatus count when open a storefile #958

Closed
wants to merge 8 commits into from
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@
public class StoreFileInfo {
private static final Logger LOG = LoggerFactory.getLogger(StoreFileInfo.class);

private FileStatus localStatus;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I like idea of caching status. When does it get invalidated?

Copy link
Author

@HuiHang-Yu HuiHang-Yu Dec 24, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Now it will be updated when reopened only because hfile will not be modified after writen i think . Is there some scenarios fileStatus will be modified?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not that I can think of.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does it need to be volatile? Will StoreFileInfo be accessed by many threads? Can it be final/created on construction?

/**
* A non-capture group, for hfiles, so that this can be embedded.
* HFiles are uuid ([0-9a-z]+). Bulk loaded hfiles has (_SeqId_[0-9]+_) has suffix.
Expand Down Expand Up @@ -107,7 +108,7 @@ public class StoreFileInfo {

private RegionCoprocessorHost coprocessorHost;

// timestamp on when the file was created, is 0 and ignored for reference or link files
// change to use the createdTimestamp of reference or link files , will it create some problem later ?
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this comment correct? The createTimeStamp is gotten once only on construction. YOu are not changing the behavior that I can see. Should createTimestamp be made final as in private final long createdTimestamp?

Copy link
Author

@HuiHang-Yu HuiHang-Yu Jan 2, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is not necessary to make createdTimestamp to be final i think .

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Its good to mark data members final if your intent is that they are not ever to change. It helps the compiler and it helps the dev reading the code later.

private long createdTimestamp;

private long size;
Expand Down Expand Up @@ -168,9 +169,8 @@ private StoreFileInfo(final Configuration conf, final FileSystem fs, final FileS
this.createdTimestamp = fileStatus.getModificationTime();
this.size = fileStatus.getLen();
} else {
FileStatus fStatus = fs.getFileStatus(initialPath);
this.createdTimestamp = fStatus.getModificationTime();
this.size = fStatus.getLen();
this.createdTimestamp = this.getFileStatus().getModificationTime();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Generally, we do not put the 'this.' in front of local method invocation.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I will remove it .

this.size = this.getFileStatus().getLen();
}
this.reference = null;
this.link = null;
Expand Down Expand Up @@ -296,7 +296,6 @@ ReaderContext createReaderContext(boolean doDropBehind, long readahead, ReaderTy
if (this.link != null) {
// HFileLink
in = new FSDataInputStreamWrapper(fs, this.link, doDropBehind, readahead);
status = this.link.getFileStatus(fs);
} else if (this.reference != null) {
// HFile Reference
Path referencePath = getReferredToFile(this.getPath());
Expand All @@ -310,11 +309,10 @@ ReaderContext createReaderContext(boolean doDropBehind, long readahead, ReaderTy
newFnfe.initCause(fnfe);
throw newFnfe;
}
status = fs.getFileStatus(referencePath);
} else {
in = new FSDataInputStreamWrapper(fs, this.getPath(), doDropBehind, readahead);
status = fs.getFileStatus(initialPath);
}
status = this.getFileStatus();
long length = status.getLen();
ReaderContextBuilder contextBuilder = new ReaderContextBuilder()
.withInputStreamWrapper(in)
Expand Down Expand Up @@ -362,21 +360,23 @@ private HDFSBlocksDistribution computeHDFSBlocksDistributionInternal(final FileS
return FSUtils.computeHDFSBlocksDistribution(fs, status, 0, status.getLen());
}
}

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems like there is a little space added. Could you please remove it?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry for my fault !

/**
* Get the {@link FileStatus} of the file referenced by this StoreFileInfo
* @param fs The current file system to use.
* @return The {@link FileStatus} of the file referenced by this StoreFileInfo
*/
public FileStatus getReferencedFileStatus(final FileSystem fs) throws IOException {
FileStatus status;
if(this.localStatus != null) {return this.localStatus;}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you think multiple threads will come in h ere at same time? Maybe it would be better to make localStatus be final and assign it on construction of this StoreFileInfo? If so, change name localStatus to status?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LocalStatus will be assigned when storeFileInfo construction is invoked . Will it be some problem happend when multiple threads will come in here at the same time ? So it is not necessary to make localstatus to be final and volatile i think .

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If so, make it final then we are explicit that it does not change during life time of a StoreFileInfo instance and we are clear about its thread safety. If done in constructor, you don't need this null check then either?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Couldn't you simplify this method if you fetched status on construction (using the bulk of this method but taking it private/internal?)?

if (this.reference != null) {
if (this.link != null) {
FileNotFoundException exToThrow = null;
for (int i = 0; i < this.link.getLocations().length; i++) {
// HFileLink Reference
try {
return link.getFileStatus(fs);
this.localStatus = link.getFileStatus(fs);
return this.localStatus;
} catch (FileNotFoundException ex) {
// try the other location
exToThrow = ex;
Expand All @@ -386,23 +386,26 @@ public FileStatus getReferencedFileStatus(final FileSystem fs) throws IOExceptio
} else {
// HFile Reference
Path referencePath = getReferredToFile(this.getPath());
status = fs.getFileStatus(referencePath);
this.localStatus = fs.getFileStatus(referencePath);
status = this.localStatus;
}
} else {
if (this.link != null) {
FileNotFoundException exToThrow = null;
for (int i = 0; i < this.link.getLocations().length; i++) {
// HFileLink
try {
return link.getFileStatus(fs);
this.localStatus = link.getFileStatus(fs);
return this.localStatus;
} catch (FileNotFoundException ex) {
// try the other location
exToThrow = ex;
}
}
throw exToThrow;
} else {
status = fs.getFileStatus(initialPath);
this.localStatus = fs.getFileStatus(initialPath);
status = this.localStatus;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Three times.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That's a good idea !

}
}
return status;
Expand Down