HDFS-13979. Review StateStoreFileSystemImpl Class. Contributed by BELUGA BEHR.
This commit is contained in:
parent
4b5b1ac3d1
commit
a05bd1288c
|
@ -24,7 +24,8 @@ import java.io.InputStreamReader;
|
|||
import java.io.OutputStreamWriter;
|
||||
import java.net.URI;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.LinkedList;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
|
@ -33,8 +34,8 @@ import org.apache.hadoop.fs.FileStatus;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Options;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys;
|
||||
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;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -50,13 +51,13 @@ public class StateStoreFileSystemImpl extends StateStoreFileBaseImpl {
|
|||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(StateStoreFileSystemImpl.class);
|
||||
|
||||
|
||||
/** Configuration keys. */
|
||||
public static final String FEDERATION_STORE_FS_PATH =
|
||||
RBFConfigKeys.FEDERATION_STORE_PREFIX + "driver.fs.path";
|
||||
|
||||
/** File system to back the State Store. */
|
||||
private FileSystem fs;
|
||||
|
||||
/** Working path in the filesystem. */
|
||||
private String workPath;
|
||||
|
||||
|
@ -141,7 +142,7 @@ public class StateStoreFileSystemImpl extends StateStoreFileBaseImpl {
|
|||
new InputStreamReader(fdis, StandardCharsets.UTF_8);
|
||||
reader = new BufferedReader(isr);
|
||||
} catch (IOException ex) {
|
||||
LOG.error("Cannot open read stream for {}", path);
|
||||
LOG.error("Cannot open read stream for {}", path, ex);
|
||||
}
|
||||
return reader;
|
||||
}
|
||||
|
@ -156,25 +157,26 @@ public class StateStoreFileSystemImpl extends StateStoreFileBaseImpl {
|
|||
new OutputStreamWriter(fdos, StandardCharsets.UTF_8);
|
||||
writer = new BufferedWriter(osw);
|
||||
} catch (IOException ex) {
|
||||
LOG.error("Cannot open write stream for {}", path);
|
||||
LOG.error("Cannot open write stream for {}", path, ex);
|
||||
}
|
||||
return writer;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<String> getChildren(String pathName) {
|
||||
List<String> ret = new LinkedList<>();
|
||||
Path path = new Path(workPath, pathName);
|
||||
try {
|
||||
FileStatus[] files = fs.listStatus(path);
|
||||
List<String> ret = new ArrayList<>(files.length);
|
||||
for (FileStatus file : files) {
|
||||
Path filePath = file.getPath();
|
||||
String fileName = filePath.getName();
|
||||
ret.add(fileName);
|
||||
}
|
||||
return ret;
|
||||
} catch (Exception e) {
|
||||
LOG.error("Cannot get children for {}", pathName, e);
|
||||
return Collections.emptyList();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue