|
@@ -27,6 +27,8 @@ import java.util.NoSuchElementException;
|
|
|
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
|
+import org.apache.hadoop.fs.BatchedRemoteIterator;
|
|
|
+import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
|
|
|
import org.apache.hadoop.fs.ContentSummary;
|
|
|
import org.apache.hadoop.fs.CreateFlag;
|
|
|
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
|
@@ -45,6 +47,7 @@ import org.apache.hadoop.hdfs.protocol.AddPathCacheDirectiveException.InvalidPoo
|
|
|
import org.apache.hadoop.hdfs.protocol.AddPathCacheDirectiveException.UnexpectedAddPathCacheDirectiveException;
|
|
|
import org.apache.hadoop.hdfs.protocol.RemovePathCacheEntryException.InvalidIdException;
|
|
|
import org.apache.hadoop.hdfs.protocol.RemovePathCacheEntryException.NoSuchIdException;
|
|
|
+import org.apache.hadoop.hdfs.protocol.RemovePathCacheEntryException.RemovePermissionDeniedException;
|
|
|
import org.apache.hadoop.hdfs.protocol.RemovePathCacheEntryException.UnexpectedRemovePathCacheEntryException;
|
|
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
|
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
|
|
@@ -58,11 +61,13 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
|
|
|
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
|
|
|
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
|
|
|
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolRequestProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.PathCacheDirectiveProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddPathCacheDirectiveErrorProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddPathCacheDirectivesRequestProto;
|
|
@@ -108,14 +113,19 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListPa
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListPathCacheEntriesRequestProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListPathCacheEntriesRequestProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListPathCacheEntriesResponseProto;
|
|
|
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsRequestProto;
|
|
|
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseElementProto;
|
|
|
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
|
|
|
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemovePathCacheEntriesRequestProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemovePathCacheEntriesResponseProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemovePathCacheEntryErrorProto;
|
|
|
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto;
|
|
@@ -1064,6 +1074,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
} else if (code == RemovePathCacheEntryErrorProto.
|
|
|
NO_SUCH_CACHED_PATH_ID_ERROR_VALUE) {
|
|
|
return new NoSuchIdException(id);
|
|
|
+ } else if (code == RemovePathCacheEntryErrorProto.
|
|
|
+ REMOVE_PERMISSION_DENIED_ERROR_VALUE) {
|
|
|
+ return new RemovePermissionDeniedException(id);
|
|
|
} else {
|
|
|
return new UnexpectedRemovePathCacheEntryException(id);
|
|
|
}
|
|
@@ -1098,32 +1111,49 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private static class BatchedPathCacheEntries
|
|
|
+ implements BatchedEntries<PathCacheEntry> {
|
|
|
+ private ListPathCacheEntriesResponseProto response;
|
|
|
+
|
|
|
+ BatchedPathCacheEntries(ListPathCacheEntriesResponseProto response) {
|
|
|
+ this.response = response;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public PathCacheEntry get(int i) {
|
|
|
+ ListPathCacheEntriesElementProto elementProto =
|
|
|
+ response.getElements(i);
|
|
|
+ return new PathCacheEntry(elementProto.getId(),
|
|
|
+ new PathCacheDirective(elementProto.getPath(),
|
|
|
+ elementProto.getPool()));
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public int size() {
|
|
|
+ return response.getElementsCount();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private class PathCacheEntriesIterator
|
|
|
- implements RemoteIterator<PathCacheEntry> {
|
|
|
- private long prevId;
|
|
|
+ extends BatchedRemoteIterator<Long, PathCacheEntry> {
|
|
|
private final String pool;
|
|
|
- private final int repliesPerRequest;
|
|
|
- private ListPathCacheEntriesResponseProto response;
|
|
|
- private int idx;
|
|
|
|
|
|
- public PathCacheEntriesIterator(long prevId, String pool,
|
|
|
- int repliesPerRequest) {
|
|
|
- this.prevId = prevId;
|
|
|
+ public PathCacheEntriesIterator(long prevKey, int maxRepliesPerRequest,
|
|
|
+ String pool) {
|
|
|
+ super(prevKey, maxRepliesPerRequest);
|
|
|
this.pool = pool;
|
|
|
- this.repliesPerRequest = repliesPerRequest;
|
|
|
- this.response = null;
|
|
|
- this.idx = -1;
|
|
|
}
|
|
|
|
|
|
- private void makeRequest() throws IOException {
|
|
|
- idx = 0;
|
|
|
- response = null;
|
|
|
+ @Override
|
|
|
+ public BatchedEntries<PathCacheEntry> makeRequest(
|
|
|
+ Long nextKey, int maxRepliesPerRequest) throws IOException {
|
|
|
+ ListPathCacheEntriesResponseProto response;
|
|
|
try {
|
|
|
ListPathCacheEntriesRequestProto req =
|
|
|
ListPathCacheEntriesRequestProto.newBuilder().
|
|
|
- setPrevId(prevId).
|
|
|
+ setPrevId(nextKey).
|
|
|
setPool(pool).
|
|
|
- setMaxReplies(repliesPerRequest).
|
|
|
+ setMaxReplies(maxRepliesPerRequest).
|
|
|
build();
|
|
|
response = rpcProxy.listPathCacheEntries(null, req);
|
|
|
if (response.getElementsCount() == 0) {
|
|
@@ -1132,45 +1162,134 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
} catch (ServiceException e) {
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
|
}
|
|
|
+ return new BatchedPathCacheEntries(response);
|
|
|
}
|
|
|
|
|
|
- private void makeRequestIfNeeded() throws IOException {
|
|
|
- if (idx == -1) {
|
|
|
- makeRequest();
|
|
|
- } else if ((response != null) && (idx >= response.getElementsCount())) {
|
|
|
- if (response.getHasMore()) {
|
|
|
- makeRequest();
|
|
|
- } else {
|
|
|
- response = null;
|
|
|
- }
|
|
|
- }
|
|
|
+ @Override
|
|
|
+ public Long elementToNextKey(PathCacheEntry element) {
|
|
|
+ return element.getEntryId();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public RemoteIterator<PathCacheEntry> listPathCacheEntries(long prevId,
|
|
|
+ String pool, int repliesPerRequest) throws IOException {
|
|
|
+ return new PathCacheEntriesIterator(prevId, repliesPerRequest, pool);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void addCachePool(CachePoolInfo info) throws IOException {
|
|
|
+ AddCachePoolRequestProto.Builder builder =
|
|
|
+ AddCachePoolRequestProto.newBuilder();
|
|
|
+ builder.setPoolName(info.getPoolName());
|
|
|
+ if (info.getOwnerName() != null) {
|
|
|
+ builder.setOwnerName(info.getOwnerName());
|
|
|
+ }
|
|
|
+ if (info.getGroupName() != null) {
|
|
|
+ builder.setGroupName(info.getGroupName());
|
|
|
+ }
|
|
|
+ if (info.getMode() != null) {
|
|
|
+ builder.setMode(info.getMode());
|
|
|
+ }
|
|
|
+ if (info.getWeight() != null) {
|
|
|
+ builder.setWeight(info.getWeight());
|
|
|
+ }
|
|
|
+ try {
|
|
|
+ rpcProxy.addCachePool(null, builder.build());
|
|
|
+ } catch (ServiceException e) {
|
|
|
+ throw ProtobufHelper.getRemoteException(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void modifyCachePool(CachePoolInfo req) throws IOException {
|
|
|
+ ModifyCachePoolRequestProto.Builder builder =
|
|
|
+ ModifyCachePoolRequestProto.newBuilder();
|
|
|
+ builder.setPoolName(req.getPoolName());
|
|
|
+ if (req.getOwnerName() != null) {
|
|
|
+ builder.setOwnerName(req.getOwnerName());
|
|
|
+ }
|
|
|
+ if (req.getGroupName() != null) {
|
|
|
+ builder.setGroupName(req.getGroupName());
|
|
|
+ }
|
|
|
+ if (req.getMode() != null) {
|
|
|
+ builder.setMode(req.getMode());
|
|
|
+ }
|
|
|
+ if (req.getWeight() != null) {
|
|
|
+ builder.setWeight(req.getWeight());
|
|
|
+ }
|
|
|
+ try {
|
|
|
+ rpcProxy.modifyCachePool(null, builder.build());
|
|
|
+ } catch (ServiceException e) {
|
|
|
+ throw ProtobufHelper.getRemoteException(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void removeCachePool(String cachePoolName) throws IOException {
|
|
|
+ try {
|
|
|
+ rpcProxy.removeCachePool(null,
|
|
|
+ RemoveCachePoolRequestProto.newBuilder().
|
|
|
+ setPoolName(cachePoolName).build());
|
|
|
+ } catch (ServiceException e) {
|
|
|
+ throw ProtobufHelper.getRemoteException(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private static class BatchedPathDirectiveEntries
|
|
|
+ implements BatchedEntries<CachePoolInfo> {
|
|
|
+ private final ListCachePoolsResponseProto proto;
|
|
|
+
|
|
|
+ public BatchedPathDirectiveEntries(ListCachePoolsResponseProto proto) {
|
|
|
+ this.proto = proto;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public CachePoolInfo get(int i) {
|
|
|
+ ListCachePoolsResponseElementProto elem = proto.getElements(i);
|
|
|
+ return new CachePoolInfo(elem.getPoolName()).
|
|
|
+ setOwnerName(elem.getOwnerName()).
|
|
|
+ setGroupName(elem.getGroupName()).
|
|
|
+ setMode(elem.getMode()).
|
|
|
+ setWeight(elem.getWeight());
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public boolean hasNext() throws IOException {
|
|
|
- makeRequestIfNeeded();
|
|
|
- return (response != null);
|
|
|
+ public int size() {
|
|
|
+ return proto.getElementsCount();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private class CachePoolIterator
|
|
|
+ extends BatchedRemoteIterator<String, CachePoolInfo> {
|
|
|
+
|
|
|
+ public CachePoolIterator(String prevKey, int maxRepliesPerRequest) {
|
|
|
+ super(prevKey, maxRepliesPerRequest);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public PathCacheEntry next() throws IOException {
|
|
|
- makeRequestIfNeeded();
|
|
|
- if (response == null) {
|
|
|
- throw new NoSuchElementException();
|
|
|
+ public BatchedEntries<CachePoolInfo> makeRequest(String prevKey,
|
|
|
+ int maxRepliesPerRequest) throws IOException {
|
|
|
+ try {
|
|
|
+ return new BatchedPathDirectiveEntries(
|
|
|
+ rpcProxy.listCachePools(null,
|
|
|
+ ListCachePoolsRequestProto.newBuilder().
|
|
|
+ setPrevPoolName(prevKey).
|
|
|
+ setMaxReplies(maxRepliesPerRequest).build()));
|
|
|
+ } catch (ServiceException e) {
|
|
|
+ throw ProtobufHelper.getRemoteException(e);
|
|
|
}
|
|
|
- ListPathCacheEntriesElementProto elementProto =
|
|
|
- response.getElements(idx);
|
|
|
- prevId = elementProto.getId();
|
|
|
- idx++;
|
|
|
- return new PathCacheEntry(elementProto.getId(),
|
|
|
- new PathCacheDirective(elementProto.getPath(),
|
|
|
- elementProto.getPool()));
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public String elementToNextKey(CachePoolInfo element) {
|
|
|
+ return element.getPoolName();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public RemoteIterator<PathCacheEntry> listPathCacheEntries(long prevId,
|
|
|
- String pool, int repliesPerRequest) throws IOException {
|
|
|
- return new PathCacheEntriesIterator(prevId, pool, repliesPerRequest);
|
|
|
+ public RemoteIterator<CachePoolInfo> listCachePools(String prevKey,
|
|
|
+ int maxRepliesPerRequest) throws IOException {
|
|
|
+ return new CachePoolIterator(prevKey, maxRepliesPerRequest);
|
|
|
}
|
|
|
}
|