-
Notifications
You must be signed in to change notification settings - Fork 751
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
[GOBBLIN-1938] preserve x bit in manifest file based copy #3804
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -255,19 +255,19 @@ private void publishFileSet(CopyEntity.DatasetAndPartition datasetAndPartition, | |
prePublish.size(), postPublish.size())); | ||
|
||
executeCommitSequence(prePublish); | ||
|
||
if (hasCopyableFiles(datasetWorkUnitStates)) { | ||
// Targets are always absolute, so we start moving from root (will skip any existing directories). | ||
HadoopUtils.renameRecursively(this.fs, datasetWriterOutputPath, new Path("/")); | ||
} else { | ||
log.info(String.format("[%s] No copyable files in dataset. Proceeding to postpublish steps.", datasetAndPartition.identifier())); | ||
} | ||
executeCommitSequence(postPublish); | ||
|
||
this.fs.delete(datasetWriterOutputPath, true); | ||
|
||
long datasetOriginTimestamp = Long.MAX_VALUE; | ||
long datasetUpstreamTimestamp = Long.MAX_VALUE; | ||
Optional<String> fileSetRoot = Optional.<String>absent(); | ||
Optional<String> fileSetRoot = Optional.absent(); | ||
|
||
for (WorkUnitState wus : datasetWorkUnitStates) { | ||
if (wus.getWorkingState() == WorkingState.SUCCESSFUL) { | ||
|
@@ -300,6 +300,10 @@ private void publishFileSet(CopyEntity.DatasetAndPartition datasetAndPartition, | |
} | ||
} | ||
|
||
// execute post publish commit steps after preserving file attributes, because some post publish step, | ||
// e.g. SetPermissionCommitStep needs to set permissions | ||
executeCommitSequence(postPublish); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We do move path from tmp to dst in this step, I believe it should happen before the set file attribute. We should make sure set file attribute preserve the right permission There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. moving from tmp(writer.output.dir actually) to dst is publish step, right? |
||
|
||
// if there are no valid values for datasetOriginTimestamp and datasetUpstreamTimestamp, use | ||
// something more readable | ||
if (Long.MAX_VALUE == datasetOriginTimestamp) { | ||
|
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 do we need this step? if dir already exist on dst and user want to change the permission for that dir, it will appear in the manifest? If dir not exist, we should directly create parent dir to be the same as source dir?
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 dir does not exist, we create it, but we create it with execute bit set. I do not want to not do that, because execute bit may be necessary to do publish.
So, in this PR, I had to add a post publish step to set the original permission.
Honestly, the usecase for this work, unsetting execute bit, seems quite rare.