HDFS-11757. Query StreamCapabilities when creating balancer's lock file. Contributed by SammiChen.
This commit is contained in:
parent
90cb5b4635
commit
83dd14aa84
@ -41,6 +41,7 @@
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FsServerDefaults;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.StreamCapabilities.StreamCapability;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.NameNodeProxies;
|
||||
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
|
||||
@ -250,8 +251,10 @@ private OutputStream checkAndMarkRunning() throws IOException {
|
||||
.setFlags(EnumSet.of(CreateFlag.CREATE))
|
||||
.build();
|
||||
|
||||
Preconditions.checkState(!fs.getFileStatus(idPath).isErasureCoded(),
|
||||
"Id File should be a replicate file");
|
||||
Preconditions.checkState(
|
||||
fsout.hasCapability(StreamCapability.HFLUSH.getValue())
|
||||
&& fsout.hasCapability(StreamCapability.HSYNC.getValue()),
|
||||
"Id lock file should support hflush and hsync");
|
||||
|
||||
// mark balancer idPath to be deleted during filesystem closure
|
||||
fs.deleteOnExit(idPath);
|
||||
|
Loading…
Reference in New Issue
Block a user