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

HDFS-11182. Update DataNode to use DatasetVolumeChecker. #168

Closed
wants to merge 5 commits into from
Closed

HDFS-11182. Update DataNode to use DatasetVolumeChecker. #168

wants to merge 5 commits into from

Conversation

arp7
Copy link
Contributor

@arp7 arp7 commented Nov 29, 2016

Preliminary patch for Jenkins runs.

* successful, add the volume here.
* @param failedVolumes set of failed volumes. If the disk check fails,
* add the volume here.
* @param semaphore semaphore used to trigger callback invocation.
Copy link
Contributor

Choose a reason for hiding this comment

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

The usage of semaphore here seems like a countUpLatch. Have you hit any problem with the existing CountDownLatch approach?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

CountDownLatch#countDown returns no value so there is no easy way to detect when the count falls to zero and the callback can be invoked (it must be invoked once only). I was using an AtomicLong to detect the 0->1 transition but it had a bug.

The semaphore approach fixes it. We still need the CountDownLatch which we can use as an event. I could have used an Object mutex instead but that would have required extra code to deal with the spurious wakeup problem which CountDownLatch does not suffer from.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think this logic can be simplified. Will post an updated patch shortly.

Copy link
Contributor

Choose a reason for hiding this comment

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

Thanks. The new logic looks good to me.

for(Iterator<FsVolumeImpl> i = volumeList.iterator(); i.hasNext(); ) {
final FsVolumeImpl fsv = i.next();
for(FsVolumeSpi vol : failedVolumes) {
FsVolumeImpl fsv = (FsVolumeImpl) vol;
Copy link
Contributor

Choose a reason for hiding this comment

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

Is it a safe cast from FsVolumeSpi to FsVolumeImpl? Can we add some log here in case the cast fail?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes. FsVolumeList is part of the fsdataset.impl package and its methods are only invoked from FsDatasetImpl so it is safe to assume that the volume is an FsVolumeImpl.

At least one existing method also makes the same assumption (see copyReplicaWithNewBlockIdAndGS).

  private File[] copyReplicaWithNewBlockIdAndGS(
      ReplicaInfo replicaInfo, String bpid, long newBlkId, long newGS)
      throws IOException {
    String blockFileName = Block.BLOCK_FILE_PREFIX + newBlkId;
    FsVolumeImpl v = (FsVolumeImpl) replicaInfo.getVolume();

Copy link
Contributor

Choose a reason for hiding this comment

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

Thanks for the explanation. Looks good to me.

@@ -124,8 +127,10 @@ public void testMinGapIsEnforcedForSyncChecks() throws Exception {

@Test(timeout=60000)
public void testMinGapIsEnforcedForASyncChecks() throws Exception {
final List<FsVolumeSpi> volumes =
Copy link
Contributor

Choose a reason for hiding this comment

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

NIT: maybe wrap the common test prep code with a helper for testMinGapIsEnforcedForSyncChecks() and testMinGapIsEnforcedForASyncChecks().

Change-Id: Idbe301392050d004461079ac38548d1e62db493f
Change-Id: Icb1c8024e974a9fb1d26e5fdb3f9df34d33e8f31
Change-Id: I9b6fe60c955c2d911bc614be3619c89cda5e99ea
@@ -1944,6 +1935,8 @@ public void shutdown() {
}
}

volumeChecker.shutdownAndWait(1, TimeUnit.SECONDS);

if (storageLocationChecker != null) {
Copy link
Contributor

Choose a reason for hiding this comment

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

The Datanode#storageLocationChecker is only needed during the datanode startup. We don't need to pass it as a parameter to DataNode constructor and keep it running during the lifetime of the datanode until datanode shutdown. This can be done as an optimization later.

Copy link
Contributor

@xiaoyuyao xiaoyuyao Dec 20, 2016

Choose a reason for hiding this comment

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

Can we reuse the synchronize version of the DatasetVolume checker for datanode startup handling? This way, we don't need to maintain two checkers for Datanode? This can be done in as a follow up if possible.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Will address it in a follow up patch.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

By the way regarding the reuse, I really wanted to do that too but it's non-trivial because the handling logic is different in both paths. It probably should have never been made different but reconciling them now is a bit of work. We can look at it in a separate Jira.

Copy link
Contributor

Choose a reason for hiding this comment

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

Agree. Let's do that in a follow up jira.

for (StorageLocation location : unhealthyLocations) {
sb.append(location + ";");
}
LOG.info(sb.toString());
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we log a warn instead of info for the failed volumes that got removed?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good point, will push an update shortly to improve the logging.

Copy link
Contributor

Choose a reason for hiding this comment

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

Thanks for fixing that.

data, (healthyVolumes, failedVolumes) -> {
LOG.info("checkDiskErrorAsync callback got {} failed volumes: {}",
failedVolumes.size(), failedVolumes);
lastDiskErrorCheck = Time.monotonicNow();
Copy link
Contributor

Choose a reason for hiding this comment

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

should this be timer.monotonicNow();

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The DataNode does not maintain a timer object right now. It is only passed to DatasetVolumeChecker during construction for unit testability of that class.

unhealthyVolumes = volumeChecker.checkAllVolumes(data);
LOG.info("checkDiskError got {} failed volumes - {}",
unhealthyVolumes.size(), unhealthyVolumes);
lastDiskErrorCheck = Time.monotonicNow();
Copy link
Contributor

Choose a reason for hiding this comment

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

should this be timer.monotonicNow();

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Same as above.

Change-Id: I9d5cb81f00ef7b0dde36be8e92887ee47a33c852
@arp7
Copy link
Contributor Author

arp7 commented Dec 20, 2016

@xiaoyuyao I pushed one more commit to improve the logging. Now we log at warn if there is a volume failure and at debug if there is no failure.

Copy link
Contributor

@xiaoyuyao xiaoyuyao left a comment

Choose a reason for hiding this comment

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

Thanks for the update. +1 with the latest change.

@aajisaka
Copy link
Member

https://jira.apache.org/jira/browse/HDFS-11182 has been fixed. Closing this as well.

@aajisaka aajisaka closed this Jul 26, 2019
shanthoosh pushed a commit to shanthoosh/hadoop that referenced this pull request Oct 15, 2019
Remove ApplicationRunner#getLocalRunner and clean up any usage examples.

Author: Xinyu Liu <xiliu@xiliu-ld.linkedin.biz>

Reviewers: Jake Maes <jmakes@apache.org>

Closes apache#168 from xinyuiscool/SAMZA-1267
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants