|
@@ -28,13 +28,14 @@ import java.util.ArrayList;
|
|
|
import java.util.Collections;
|
|
|
import java.util.List;
|
|
|
|
|
|
+import org.apache.hadoop.classification.VisibleForTesting;
|
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
+import org.apache.hadoop.fs.FileUtil;
|
|
|
import org.apache.hadoop.fs.Options;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
-import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreDriver;
|
|
|
import org.apache.hadoop.hdfs.server.federation.store.records.BaseRecord;
|
|
@@ -82,17 +83,8 @@ public class StateStoreFileSystemImpl extends StateStoreFileBaseImpl {
|
|
|
@Override
|
|
|
protected boolean rename(String src, String dst) {
|
|
|
try {
|
|
|
- if (fs instanceof DistributedFileSystem) {
|
|
|
- DistributedFileSystem dfs = (DistributedFileSystem)fs;
|
|
|
- dfs.rename(new Path(src), new Path(dst), Options.Rename.OVERWRITE);
|
|
|
- return true;
|
|
|
- } else {
|
|
|
- // Replace should be atomic but not available
|
|
|
- if (fs.exists(new Path(dst))) {
|
|
|
- fs.delete(new Path(dst), true);
|
|
|
- }
|
|
|
- return fs.rename(new Path(src), new Path(dst));
|
|
|
- }
|
|
|
+ FileUtil.rename(fs, new Path(src), new Path(dst), Options.Rename.OVERWRITE);
|
|
|
+ return true;
|
|
|
} catch (Exception e) {
|
|
|
LOG.error("Cannot rename {} to {}", src, dst, e);
|
|
|
return false;
|
|
@@ -148,7 +140,8 @@ public class StateStoreFileSystemImpl extends StateStoreFileBaseImpl {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- protected <T extends BaseRecord> BufferedWriter getWriter(String pathName) {
|
|
|
+ @VisibleForTesting
|
|
|
+ public <T extends BaseRecord> BufferedWriter getWriter(String pathName) {
|
|
|
BufferedWriter writer = null;
|
|
|
Path path = new Path(pathName);
|
|
|
try {
|