|
@@ -18,7 +18,6 @@
|
|
|
package org.apache.hadoop.hdfs.protocolPB;
|
|
|
|
|
|
import java.io.Closeable;
|
|
|
-import java.io.FileNotFoundException;
|
|
|
import java.io.IOException;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.EnumSet;
|
|
@@ -33,12 +32,9 @@ import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
|
|
|
import org.apache.hadoop.fs.CacheFlag;
|
|
|
import org.apache.hadoop.fs.ContentSummary;
|
|
|
import org.apache.hadoop.fs.CreateFlag;
|
|
|
-import org.apache.hadoop.fs.FileAlreadyExistsException;
|
|
|
import org.apache.hadoop.fs.FsServerDefaults;
|
|
|
import org.apache.hadoop.fs.Options.Rename;
|
|
|
-import org.apache.hadoop.fs.ParentNotDirectoryException;
|
|
|
import org.apache.hadoop.fs.StorageType;
|
|
|
-import org.apache.hadoop.fs.UnresolvedLinkException;
|
|
|
import org.apache.hadoop.fs.XAttr;
|
|
|
import org.apache.hadoop.fs.XAttrSetFlag;
|
|
|
import org.apache.hadoop.fs.permission.AclEntry;
|
|
@@ -46,7 +42,6 @@ import org.apache.hadoop.fs.permission.AclStatus;
|
|
|
import org.apache.hadoop.fs.permission.FsAction;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.hdfs.inotify.EventBatchList;
|
|
|
-import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
|
|
|
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
|
|
|
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
|
|
|
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
|
|
@@ -54,7 +49,6 @@ import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
|
|
|
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
|
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
|
|
|
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
|
@@ -67,7 +61,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
|
import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
|
|
|
import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
|
|
|
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
|
|
@@ -171,8 +164,6 @@ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
|
|
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
|
|
|
-import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
|
|
|
import org.apache.hadoop.io.EnumSetWritable;
|
|
|
import org.apache.hadoop.io.Text;
|
|
@@ -181,7 +172,6 @@ import org.apache.hadoop.ipc.ProtocolMetaInterface;
|
|
|
import org.apache.hadoop.ipc.ProtocolTranslator;
|
|
|
import org.apache.hadoop.ipc.RPC;
|
|
|
import org.apache.hadoop.ipc.RpcClientUtil;
|
|
|
-import org.apache.hadoop.security.AccessControlException;
|
|
|
import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
|
|
|
import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
|
|
|
import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProto;
|
|
@@ -206,37 +196,37 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
|
|
|
final private ClientNamenodeProtocolPB rpcProxy;
|
|
|
|
|
|
- static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST =
|
|
|
- GetServerDefaultsRequestProto.newBuilder().build();
|
|
|
+ static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST =
|
|
|
+ GetServerDefaultsRequestProto.newBuilder().build();
|
|
|
|
|
|
private final static GetFsStatusRequestProto VOID_GET_FSSTATUS_REQUEST =
|
|
|
- GetFsStatusRequestProto.newBuilder().build();
|
|
|
+ GetFsStatusRequestProto.newBuilder().build();
|
|
|
|
|
|
private final static SaveNamespaceRequestProto VOID_SAVE_NAMESPACE_REQUEST =
|
|
|
SaveNamespaceRequestProto.newBuilder().build();
|
|
|
|
|
|
- private final static RollEditsRequestProto VOID_ROLLEDITS_REQUEST =
|
|
|
+ private final static RollEditsRequestProto VOID_ROLLEDITS_REQUEST =
|
|
|
RollEditsRequestProto.getDefaultInstance();
|
|
|
|
|
|
private final static RefreshNodesRequestProto VOID_REFRESH_NODES_REQUEST =
|
|
|
- RefreshNodesRequestProto.newBuilder().build();
|
|
|
+ RefreshNodesRequestProto.newBuilder().build();
|
|
|
|
|
|
private final static FinalizeUpgradeRequestProto
|
|
|
- VOID_FINALIZE_UPGRADE_REQUEST =
|
|
|
+ VOID_FINALIZE_UPGRADE_REQUEST =
|
|
|
FinalizeUpgradeRequestProto.newBuilder().build();
|
|
|
|
|
|
private final static GetDataEncryptionKeyRequestProto
|
|
|
- VOID_GET_DATA_ENCRYPTIONKEY_REQUEST =
|
|
|
+ VOID_GET_DATA_ENCRYPTIONKEY_REQUEST =
|
|
|
GetDataEncryptionKeyRequestProto.newBuilder().build();
|
|
|
|
|
|
private final static GetStoragePoliciesRequestProto
|
|
|
- VOID_GET_STORAGE_POLICIES_REQUEST =
|
|
|
+ VOID_GET_STORAGE_POLICIES_REQUEST =
|
|
|
GetStoragePoliciesRequestProto.newBuilder().build();
|
|
|
|
|
|
public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) {
|
|
|
rpcProxy = proxy;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public void close() {
|
|
|
RPC.stopProxy(rpcProxy);
|
|
@@ -244,8 +234,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
|
|
|
@Override
|
|
|
public LocatedBlocks getBlockLocations(String src, long offset, long length)
|
|
|
- throws AccessControlException, FileNotFoundException,
|
|
|
- UnresolvedLinkException, IOException {
|
|
|
+ throws IOException {
|
|
|
GetBlockLocationsRequestProto req = GetBlockLocationsRequestProto
|
|
|
.newBuilder()
|
|
|
.setSrc(src)
|
|
@@ -256,7 +245,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
GetBlockLocationsResponseProto resp = rpcProxy.getBlockLocations(null,
|
|
|
req);
|
|
|
return resp.hasLocations() ?
|
|
|
- PBHelperClient.convert(resp.getLocations()) : null;
|
|
|
+ PBHelperClient.convert(resp.getLocations()) : null;
|
|
|
} catch (ServiceException e) {
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
|
}
|
|
@@ -276,13 +265,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
@Override
|
|
|
public HdfsFileStatus create(String src, FsPermission masked,
|
|
|
String clientName, EnumSetWritable<CreateFlag> flag,
|
|
|
- boolean createParent, short replication, long blockSize,
|
|
|
+ boolean createParent, short replication, long blockSize,
|
|
|
CryptoProtocolVersion[] supportedVersions)
|
|
|
- throws AccessControlException, AlreadyBeingCreatedException,
|
|
|
- DSQuotaExceededException, FileAlreadyExistsException,
|
|
|
- FileNotFoundException, NSQuotaExceededException,
|
|
|
- ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
|
|
|
- IOException {
|
|
|
+ throws IOException {
|
|
|
CreateRequestProto.Builder builder = CreateRequestProto.newBuilder()
|
|
|
.setSrc(src)
|
|
|
.setMasked(PBHelperClient.convert(masked))
|
|
@@ -291,7 +276,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
.setCreateParent(createParent)
|
|
|
.setReplication(replication)
|
|
|
.setBlockSize(blockSize);
|
|
|
- builder.addAllCryptoProtocolVersion(PBHelperClient.convert(supportedVersions));
|
|
|
+ builder.addAllCryptoProtocolVersion(
|
|
|
+ PBHelperClient.convert(supportedVersions));
|
|
|
CreateRequestProto req = builder.build();
|
|
|
try {
|
|
|
CreateResponseProto res = rpcProxy.create(null, req);
|
|
@@ -304,7 +290,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
|
|
|
@Override
|
|
|
public boolean truncate(String src, long newLength, String clientName)
|
|
|
- throws IOException, UnresolvedLinkException {
|
|
|
+ throws IOException {
|
|
|
TruncateRequestProto req = TruncateRequestProto.newBuilder()
|
|
|
.setSrc(src)
|
|
|
.setNewLength(newLength)
|
|
@@ -319,18 +305,17 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
|
|
|
@Override
|
|
|
public LastBlockWithStatus append(String src, String clientName,
|
|
|
- EnumSetWritable<CreateFlag> flag) throws AccessControlException,
|
|
|
- DSQuotaExceededException, FileNotFoundException, SafeModeException,
|
|
|
- UnresolvedLinkException, IOException {
|
|
|
+ EnumSetWritable<CreateFlag> flag) throws IOException {
|
|
|
AppendRequestProto req = AppendRequestProto.newBuilder().setSrc(src)
|
|
|
- .setClientName(clientName).setFlag(PBHelperClient.convertCreateFlag(flag))
|
|
|
+ .setClientName(clientName).setFlag(
|
|
|
+ PBHelperClient.convertCreateFlag(flag))
|
|
|
.build();
|
|
|
try {
|
|
|
AppendResponseProto res = rpcProxy.append(null, req);
|
|
|
LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient
|
|
|
.convert(res.getBlock()) : null;
|
|
|
- HdfsFileStatus stat = (res.hasStat()) ? PBHelperClient.convert(res.getStat())
|
|
|
- : null;
|
|
|
+ HdfsFileStatus stat = (res.hasStat()) ?
|
|
|
+ PBHelperClient.convert(res.getStat()) : null;
|
|
|
return new LastBlockWithStatus(lastBlock, stat);
|
|
|
} catch (ServiceException e) {
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
@@ -339,9 +324,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
|
|
|
@Override
|
|
|
public boolean setReplication(String src, short replication)
|
|
|
- throws AccessControlException, DSQuotaExceededException,
|
|
|
- FileNotFoundException, SafeModeException, UnresolvedLinkException,
|
|
|
- IOException {
|
|
|
+ throws IOException {
|
|
|
SetReplicationRequestProto req = SetReplicationRequestProto.newBuilder()
|
|
|
.setSrc(src)
|
|
|
.setReplication(replication)
|
|
@@ -355,8 +338,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
|
|
|
@Override
|
|
|
public void setPermission(String src, FsPermission permission)
|
|
|
- throws AccessControlException, FileNotFoundException, SafeModeException,
|
|
|
- UnresolvedLinkException, IOException {
|
|
|
+ throws IOException {
|
|
|
SetPermissionRequestProto req = SetPermissionRequestProto.newBuilder()
|
|
|
.setSrc(src)
|
|
|
.setPermission(PBHelperClient.convert(permission))
|
|
@@ -370,14 +352,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
|
|
|
@Override
|
|
|
public void setOwner(String src, String username, String groupname)
|
|
|
- throws AccessControlException, FileNotFoundException, SafeModeException,
|
|
|
- UnresolvedLinkException, IOException {
|
|
|
+ throws IOException {
|
|
|
SetOwnerRequestProto.Builder req = SetOwnerRequestProto.newBuilder()
|
|
|
.setSrc(src);
|
|
|
if (username != null)
|
|
|
- req.setUsername(username);
|
|
|
+ req.setUsername(username);
|
|
|
if (groupname != null)
|
|
|
- req.setGroupname(groupname);
|
|
|
+ req.setGroupname(groupname);
|
|
|
try {
|
|
|
rpcProxy.setOwner(null, req.build());
|
|
|
} catch (ServiceException e) {
|
|
@@ -387,28 +368,24 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
|
|
|
@Override
|
|
|
public void abandonBlock(ExtendedBlock b, long fileId, String src,
|
|
|
- String holder) throws AccessControlException, FileNotFoundException,
|
|
|
- UnresolvedLinkException, IOException {
|
|
|
+ String holder) throws IOException {
|
|
|
AbandonBlockRequestProto req = AbandonBlockRequestProto.newBuilder()
|
|
|
.setB(PBHelperClient.convert(b)).setSrc(src).setHolder(holder)
|
|
|
- .setFileId(fileId).build();
|
|
|
+ .setFileId(fileId).build();
|
|
|
try {
|
|
|
rpcProxy.abandonBlock(null, req);
|
|
|
} catch (ServiceException e) {
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public LocatedBlock addBlock(String src, String clientName,
|
|
|
ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId,
|
|
|
- String[] favoredNodes)
|
|
|
- throws AccessControlException, FileNotFoundException,
|
|
|
- NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
|
|
|
- IOException {
|
|
|
+ String[] favoredNodes) throws IOException {
|
|
|
AddBlockRequestProto.Builder req = AddBlockRequestProto.newBuilder()
|
|
|
.setSrc(src).setClientName(clientName).setFileId(fileId);
|
|
|
- if (previous != null)
|
|
|
+ if (previous != null)
|
|
|
req.setPrevious(PBHelperClient.convert(previous));
|
|
|
if (excludeNodes != null)
|
|
|
req.addAllExcludeNodes(PBHelperClient.convert(excludeNodes));
|
|
@@ -425,10 +402,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
@Override
|
|
|
public LocatedBlock getAdditionalDatanode(String src, long fileId,
|
|
|
ExtendedBlock blk, DatanodeInfo[] existings, String[] existingStorageIDs,
|
|
|
- DatanodeInfo[] excludes,
|
|
|
- int numAdditionalNodes, String clientName) throws AccessControlException,
|
|
|
- FileNotFoundException, SafeModeException, UnresolvedLinkException,
|
|
|
- IOException {
|
|
|
+ DatanodeInfo[] excludes, int numAdditionalNodes, String clientName)
|
|
|
+ throws IOException {
|
|
|
GetAdditionalDatanodeRequestProto req = GetAdditionalDatanodeRequestProto
|
|
|
.newBuilder()
|
|
|
.setSrc(src)
|
|
@@ -450,9 +425,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
|
|
|
@Override
|
|
|
public boolean complete(String src, String clientName,
|
|
|
- ExtendedBlock last, long fileId)
|
|
|
- throws AccessControlException, FileNotFoundException, SafeModeException,
|
|
|
- UnresolvedLinkException, IOException {
|
|
|
+ ExtendedBlock last, long fileId) throws IOException {
|
|
|
CompleteRequestProto.Builder req = CompleteRequestProto.newBuilder()
|
|
|
.setSrc(src)
|
|
|
.setClientName(clientName)
|
|
@@ -469,7 +442,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
@Override
|
|
|
public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
|
|
|
ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder()
|
|
|
- .addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlock(blocks)))
|
|
|
+ .addAllBlocks(Arrays.asList(
|
|
|
+ PBHelperClient.convertLocatedBlock(blocks)))
|
|
|
.build();
|
|
|
try {
|
|
|
rpcProxy.reportBadBlocks(null, req);
|
|
@@ -479,8 +453,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public boolean rename(String src, String dst) throws UnresolvedLinkException,
|
|
|
- IOException {
|
|
|
+ public boolean rename(String src, String dst) throws IOException {
|
|
|
RenameRequestProto req = RenameRequestProto.newBuilder()
|
|
|
.setSrc(src)
|
|
|
.setDst(dst).build();
|
|
@@ -490,14 +463,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
|
|
|
@Override
|
|
|
public void rename2(String src, String dst, Rename... options)
|
|
|
- throws AccessControlException, DSQuotaExceededException,
|
|
|
- FileAlreadyExistsException, FileNotFoundException,
|
|
|
- NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
|
|
|
- UnresolvedLinkException, IOException {
|
|
|
+ throws IOException {
|
|
|
boolean overwrite = false;
|
|
|
if (options != null) {
|
|
|
for (Rename option : options) {
|
|
@@ -519,8 +489,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void concat(String trg, String[] srcs) throws IOException,
|
|
|
- UnresolvedLinkException {
|
|
|
+ public void concat(String trg, String[] srcs) throws IOException {
|
|
|
ConcatRequestProto req = ConcatRequestProto.newBuilder().
|
|
|
setTrg(trg).
|
|
|
addAllSrcs(Arrays.asList(srcs)).build();
|
|
@@ -533,10 +502,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
|
|
|
|
|
|
@Override
|
|
|
- public boolean delete(String src, boolean recursive)
|
|
|
- throws AccessControlException, FileNotFoundException, SafeModeException,
|
|
|
- UnresolvedLinkException, IOException {
|
|
|
- DeleteRequestProto req = DeleteRequestProto.newBuilder().setSrc(src).setRecursive(recursive).build();
|
|
|
+ public boolean delete(String src, boolean recursive) throws IOException {
|
|
|
+ DeleteRequestProto req = DeleteRequestProto.newBuilder().setSrc(src)
|
|
|
+ .setRecursive(recursive).build();
|
|
|
try {
|
|
|
return rpcProxy.delete(null, req).getResult();
|
|
|
} catch (ServiceException e) {
|
|
@@ -546,10 +514,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
|
|
|
@Override
|
|
|
public boolean mkdirs(String src, FsPermission masked, boolean createParent)
|
|
|
- throws AccessControlException, FileAlreadyExistsException,
|
|
|
- FileNotFoundException, NSQuotaExceededException,
|
|
|
- ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
|
|
|
- IOException {
|
|
|
+ throws IOException {
|
|
|
MkdirsRequestProto req = MkdirsRequestProto.newBuilder()
|
|
|
.setSrc(src)
|
|
|
.setMasked(PBHelperClient.convert(masked))
|
|
@@ -564,15 +529,14 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
|
|
|
@Override
|
|
|
public DirectoryListing getListing(String src, byte[] startAfter,
|
|
|
- boolean needLocation) throws AccessControlException,
|
|
|
- FileNotFoundException, UnresolvedLinkException, IOException {
|
|
|
+ boolean needLocation) throws IOException {
|
|
|
GetListingRequestProto req = GetListingRequestProto.newBuilder()
|
|
|
.setSrc(src)
|
|
|
.setStartAfter(ByteString.copyFrom(startAfter))
|
|
|
.setNeedLocation(needLocation).build();
|
|
|
try {
|
|
|
GetListingResponseProto result = rpcProxy.getListing(null, req);
|
|
|
-
|
|
|
+
|
|
|
if (result.hasDirList()) {
|
|
|
return PBHelperClient.convert(result.getDirList());
|
|
|
}
|
|
@@ -583,8 +547,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void renewLease(String clientName) throws AccessControlException,
|
|
|
- IOException {
|
|
|
+ public void renewLease(String clientName) throws IOException {
|
|
|
RenewLeaseRequestProto req = RenewLeaseRequestProto.newBuilder()
|
|
|
.setClientName(clientName).build();
|
|
|
try {
|
|
@@ -604,7 +567,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
return rpcProxy.recoverLease(null, req).getResult();
|
|
|
} catch (ServiceException e) {
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
|
- }
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -632,22 +595,22 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public DatanodeStorageReport[] getDatanodeStorageReport(DatanodeReportType type)
|
|
|
- throws IOException {
|
|
|
+ public DatanodeStorageReport[] getDatanodeStorageReport(
|
|
|
+ DatanodeReportType type) throws IOException {
|
|
|
final GetDatanodeStorageReportRequestProto req
|
|
|
= GetDatanodeStorageReportRequestProto.newBuilder()
|
|
|
- .setType(PBHelperClient.convert(type)).build();
|
|
|
+ .setType(PBHelperClient.convert(type)).build();
|
|
|
try {
|
|
|
return PBHelperClient.convertDatanodeStorageReports(
|
|
|
- rpcProxy.getDatanodeStorageReport(null, req).getDatanodeStorageReportsList());
|
|
|
+ rpcProxy.getDatanodeStorageReport(null, req)
|
|
|
+ .getDatanodeStorageReportsList());
|
|
|
} catch (ServiceException e) {
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public long getPreferredBlockSize(String filename) throws IOException,
|
|
|
- UnresolvedLinkException {
|
|
|
+ public long getPreferredBlockSize(String filename) throws IOException {
|
|
|
GetPreferredBlockSizeRequestProto req = GetPreferredBlockSizeRequestProto
|
|
|
.newBuilder()
|
|
|
.setFilename(filename)
|
|
@@ -660,9 +623,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException {
|
|
|
+ public boolean setSafeMode(SafeModeAction action, boolean isChecked)
|
|
|
+ throws IOException {
|
|
|
SetSafeModeRequestProto req = SetSafeModeRequestProto.newBuilder()
|
|
|
- .setAction(PBHelperClient.convert(action)).setChecked(isChecked).build();
|
|
|
+ .setAction(PBHelperClient.convert(action))
|
|
|
+ .setChecked(isChecked).build();
|
|
|
try {
|
|
|
return rpcProxy.setSafeMode(null, req).getResult();
|
|
|
} catch (ServiceException e) {
|
|
@@ -671,16 +636,16 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void saveNamespace() throws AccessControlException, IOException {
|
|
|
+ public void saveNamespace() throws IOException {
|
|
|
try {
|
|
|
rpcProxy.saveNamespace(null, VOID_SAVE_NAMESPACE_REQUEST);
|
|
|
} catch (ServiceException e) {
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
- public long rollEdits() throws AccessControlException, IOException {
|
|
|
+ public long rollEdits() throws IOException {
|
|
|
try {
|
|
|
RollEditsResponseProto resp = rpcProxy.rollEdits(null,
|
|
|
VOID_ROLLEDITS_REQUEST);
|
|
@@ -691,8 +656,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public boolean restoreFailedStorage(String arg)
|
|
|
- throws AccessControlException, IOException{
|
|
|
+ public boolean restoreFailedStorage(String arg) throws IOException{
|
|
|
RestoreFailedStorageRequestProto req = RestoreFailedStorageRequestProto
|
|
|
.newBuilder()
|
|
|
.setArg(arg).build();
|
|
@@ -722,11 +686,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
|
|
|
+ public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
|
|
|
+ throws IOException {
|
|
|
final RollingUpgradeRequestProto r = RollingUpgradeRequestProto.newBuilder()
|
|
|
.setAction(PBHelperClient.convert(action)).build();
|
|
|
try {
|
|
|
- final RollingUpgradeResponseProto proto = rpcProxy.rollingUpgrade(null, r);
|
|
|
+ final RollingUpgradeResponseProto proto =
|
|
|
+ rpcProxy.rollingUpgrade(null, r);
|
|
|
if (proto.hasRollingUpgradeInfo()) {
|
|
|
return PBHelperClient.convert(proto.getRollingUpgradeInfo());
|
|
|
}
|
|
@@ -739,9 +705,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
@Override
|
|
|
public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
|
|
|
throws IOException {
|
|
|
- ListCorruptFileBlocksRequestProto.Builder req =
|
|
|
- ListCorruptFileBlocksRequestProto.newBuilder().setPath(path);
|
|
|
- if (cookie != null)
|
|
|
+ ListCorruptFileBlocksRequestProto.Builder req =
|
|
|
+ ListCorruptFileBlocksRequestProto.newBuilder().setPath(path);
|
|
|
+ if (cookie != null)
|
|
|
req.setCookie(cookie);
|
|
|
try {
|
|
|
return PBHelperClient.convert(
|
|
@@ -764,8 +730,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
|
|
|
- FileNotFoundException, UnresolvedLinkException, IOException {
|
|
|
+ public HdfsFileStatus getFileInfo(String src) throws IOException {
|
|
|
GetFileInfoRequestProto req = GetFileInfoRequestProto.newBuilder()
|
|
|
.setSrc(src).build();
|
|
|
try {
|
|
@@ -777,23 +742,21 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public HdfsFileStatus getFileLinkInfo(String src)
|
|
|
- throws AccessControlException, UnresolvedLinkException, IOException {
|
|
|
+ public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
|
|
|
GetFileLinkInfoRequestProto req = GetFileLinkInfoRequestProto.newBuilder()
|
|
|
.setSrc(src).build();
|
|
|
try {
|
|
|
GetFileLinkInfoResponseProto result = rpcProxy.getFileLinkInfo(null, req);
|
|
|
- return result.hasFs() ?
|
|
|
- PBHelperClient.convert(rpcProxy.getFileLinkInfo(null, req).getFs()) : null;
|
|
|
+ return result.hasFs() ?
|
|
|
+ PBHelperClient.convert(rpcProxy.getFileLinkInfo(null, req).getFs()) :
|
|
|
+ null;
|
|
|
} catch (ServiceException e) {
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public ContentSummary getContentSummary(String path)
|
|
|
- throws AccessControlException, FileNotFoundException,
|
|
|
- UnresolvedLinkException, IOException {
|
|
|
+ public ContentSummary getContentSummary(String path) throws IOException {
|
|
|
GetContentSummaryRequestProto req = GetContentSummaryRequestProto
|
|
|
.newBuilder()
|
|
|
.setPath(path)
|
|
@@ -808,9 +771,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
|
|
|
@Override
|
|
|
public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
|
|
|
- StorageType type)
|
|
|
- throws AccessControlException, FileNotFoundException,
|
|
|
- UnresolvedLinkException, IOException {
|
|
|
+ StorageType type) throws IOException {
|
|
|
final SetQuotaRequestProto.Builder builder
|
|
|
= SetQuotaRequestProto.newBuilder()
|
|
|
.setPath(path)
|
|
@@ -829,12 +790,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
|
|
|
@Override
|
|
|
public void fsync(String src, long fileId, String client,
|
|
|
- long lastBlockLength)
|
|
|
- throws AccessControlException, FileNotFoundException,
|
|
|
- UnresolvedLinkException, IOException {
|
|
|
+ long lastBlockLength) throws IOException {
|
|
|
FsyncRequestProto req = FsyncRequestProto.newBuilder().setSrc(src)
|
|
|
.setClient(client).setLastBlockLength(lastBlockLength)
|
|
|
- .setFileId(fileId).build();
|
|
|
+ .setFileId(fileId).build();
|
|
|
try {
|
|
|
rpcProxy.fsync(null, req);
|
|
|
} catch (ServiceException e) {
|
|
@@ -843,9 +802,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void setTimes(String src, long mtime, long atime)
|
|
|
- throws AccessControlException, FileNotFoundException,
|
|
|
- UnresolvedLinkException, IOException {
|
|
|
+ public void setTimes(String src, long mtime, long atime) throws IOException {
|
|
|
SetTimesRequestProto req = SetTimesRequestProto.newBuilder()
|
|
|
.setSrc(src)
|
|
|
.setMtime(mtime)
|
|
@@ -860,10 +817,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
|
|
|
@Override
|
|
|
public void createSymlink(String target, String link, FsPermission dirPerm,
|
|
|
- boolean createParent) throws AccessControlException,
|
|
|
- FileAlreadyExistsException, FileNotFoundException,
|
|
|
- ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
|
|
|
- IOException {
|
|
|
+ boolean createParent) throws IOException {
|
|
|
CreateSymlinkRequestProto req = CreateSymlinkRequestProto.newBuilder()
|
|
|
.setTarget(target)
|
|
|
.setLink(link)
|
|
@@ -878,8 +832,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public String getLinkTarget(String path) throws AccessControlException,
|
|
|
- FileNotFoundException, IOException {
|
|
|
+ public String getLinkTarget(String path) throws IOException {
|
|
|
GetLinkTargetRequestProto req = GetLinkTargetRequestProto.newBuilder()
|
|
|
.setPath(path).build();
|
|
|
try {
|
|
@@ -908,7 +861,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
|
|
|
@Override
|
|
|
public void updatePipeline(String clientName, ExtendedBlock oldBlock,
|
|
|
- ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs) throws IOException {
|
|
|
+ ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs)
|
|
|
+ throws IOException {
|
|
|
UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder()
|
|
|
.setClientName(clientName)
|
|
|
.setOldBlock(PBHelperClient.convert(oldBlock))
|
|
@@ -931,9 +885,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
.setRenewer(renewer == null ? "" : renewer.toString())
|
|
|
.build();
|
|
|
try {
|
|
|
- GetDelegationTokenResponseProto resp = rpcProxy.getDelegationToken(null, req);
|
|
|
- return resp.hasToken() ? PBHelperClient.convertDelegationToken(resp.getToken())
|
|
|
- : null;
|
|
|
+ GetDelegationTokenResponseProto resp =
|
|
|
+ rpcProxy.getDelegationToken(null, req);
|
|
|
+ return resp.hasToken() ?
|
|
|
+ PBHelperClient.convertDelegationToken(resp.getToken()) : null;
|
|
|
} catch (ServiceException e) {
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
|
}
|
|
@@ -942,9 +897,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
@Override
|
|
|
public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
|
|
|
throws IOException {
|
|
|
- RenewDelegationTokenRequestProto req = RenewDelegationTokenRequestProto.newBuilder().
|
|
|
- setToken(PBHelperClient.convert(token)).
|
|
|
- build();
|
|
|
+ RenewDelegationTokenRequestProto req =
|
|
|
+ RenewDelegationTokenRequestProto.newBuilder().
|
|
|
+ setToken(PBHelperClient.convert(token)).
|
|
|
+ build();
|
|
|
try {
|
|
|
return rpcProxy.renewDelegationToken(null, req).getNewExpiryTime();
|
|
|
} catch (ServiceException e) {
|
|
@@ -968,9 +924,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
|
|
|
@Override
|
|
|
public void setBalancerBandwidth(long bandwidth) throws IOException {
|
|
|
- SetBalancerBandwidthRequestProto req = SetBalancerBandwidthRequestProto.newBuilder()
|
|
|
- .setBandwidth(bandwidth)
|
|
|
- .build();
|
|
|
+ SetBalancerBandwidthRequestProto req =
|
|
|
+ SetBalancerBandwidthRequestProto.newBuilder()
|
|
|
+ .setBandwidth(bandwidth)
|
|
|
+ .build();
|
|
|
try {
|
|
|
rpcProxy.setBalancerBandwidth(null, req);
|
|
|
} catch (ServiceException e) {
|
|
@@ -984,23 +941,22 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
ClientNamenodeProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
|
|
|
RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), methodName);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public DataEncryptionKey getDataEncryptionKey() throws IOException {
|
|
|
try {
|
|
|
GetDataEncryptionKeyResponseProto rsp = rpcProxy.getDataEncryptionKey(
|
|
|
null, VOID_GET_DATA_ENCRYPTIONKEY_REQUEST);
|
|
|
- return rsp.hasDataEncryptionKey() ?
|
|
|
+ return rsp.hasDataEncryptionKey() ?
|
|
|
PBHelperClient.convert(rsp.getDataEncryptionKey()) : null;
|
|
|
} catch (ServiceException e) {
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
|
|
|
@Override
|
|
|
- public boolean isFileClosed(String src) throws AccessControlException,
|
|
|
- FileNotFoundException, UnresolvedLinkException, IOException {
|
|
|
+ public boolean isFileClosed(String src) throws IOException {
|
|
|
IsFileClosedRequestProto req = IsFileClosedRequestProto.newBuilder()
|
|
|
.setSrc(src).build();
|
|
|
try {
|
|
@@ -1030,7 +986,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public void deleteSnapshot(String snapshotRoot, String snapshotName)
|
|
|
throws IOException {
|
|
@@ -1042,7 +998,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public void allowSnapshot(String snapshotRoot) throws IOException {
|
|
|
AllowSnapshotRequestProto req = AllowSnapshotRequestProto.newBuilder()
|
|
@@ -1081,12 +1037,12 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
@Override
|
|
|
public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
|
|
|
throws IOException {
|
|
|
- GetSnapshottableDirListingRequestProto req =
|
|
|
+ GetSnapshottableDirListingRequestProto req =
|
|
|
GetSnapshottableDirListingRequestProto.newBuilder().build();
|
|
|
try {
|
|
|
GetSnapshottableDirListingResponseProto result = rpcProxy
|
|
|
.getSnapshottableDirListing(null, req);
|
|
|
-
|
|
|
+
|
|
|
if (result.hasSnapshottableDirList()) {
|
|
|
return PBHelperClient.convert(result.getSnapshottableDirList());
|
|
|
}
|
|
@@ -1103,9 +1059,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
.newBuilder().setSnapshotRoot(snapshotRoot)
|
|
|
.setFromSnapshot(fromSnapshot).setToSnapshot(toSnapshot).build();
|
|
|
try {
|
|
|
- GetSnapshotDiffReportResponseProto result =
|
|
|
+ GetSnapshotDiffReportResponseProto result =
|
|
|
rpcProxy.getSnapshotDiffReport(null, req);
|
|
|
-
|
|
|
+
|
|
|
return PBHelperClient.convert(result.getDiffReport());
|
|
|
} catch (ServiceException e) {
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
@@ -1174,7 +1130,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
public int size() {
|
|
|
return response.getElementsCount();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public boolean hasMore() {
|
|
|
return response.getHasMore();
|
|
@@ -1182,19 +1138,18 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public BatchedEntries<CacheDirectiveEntry>
|
|
|
- listCacheDirectives(long prevId,
|
|
|
- CacheDirectiveInfo filter) throws IOException {
|
|
|
+ public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(long prevId,
|
|
|
+ CacheDirectiveInfo filter) throws IOException {
|
|
|
if (filter == null) {
|
|
|
filter = new CacheDirectiveInfo.Builder().build();
|
|
|
}
|
|
|
try {
|
|
|
return new BatchedCacheEntries(
|
|
|
- rpcProxy.listCacheDirectives(null,
|
|
|
- ListCacheDirectivesRequestProto.newBuilder().
|
|
|
- setPrevId(prevId).
|
|
|
- setFilter(PBHelperClient.convert(filter)).
|
|
|
- build()));
|
|
|
+ rpcProxy.listCacheDirectives(null,
|
|
|
+ ListCacheDirectivesRequestProto.newBuilder().
|
|
|
+ setPrevId(prevId).
|
|
|
+ setFilter(PBHelperClient.convert(filter)).
|
|
|
+ build()));
|
|
|
} catch (ServiceException e) {
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
|
}
|
|
@@ -1202,7 +1157,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
|
|
|
@Override
|
|
|
public void addCachePool(CachePoolInfo info) throws IOException {
|
|
|
- AddCachePoolRequestProto.Builder builder =
|
|
|
+ AddCachePoolRequestProto.Builder builder =
|
|
|
AddCachePoolRequestProto.newBuilder();
|
|
|
builder.setInfo(PBHelperClient.convert(info));
|
|
|
try {
|
|
@@ -1214,7 +1169,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
|
|
|
@Override
|
|
|
public void modifyCachePool(CachePoolInfo req) throws IOException {
|
|
|
- ModifyCachePoolRequestProto.Builder builder =
|
|
|
+ ModifyCachePoolRequestProto.Builder builder =
|
|
|
ModifyCachePoolRequestProto.newBuilder();
|
|
|
builder.setInfo(PBHelperClient.convert(req));
|
|
|
try {
|
|
@@ -1227,22 +1182,22 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
@Override
|
|
|
public void removeCachePool(String cachePoolName) throws IOException {
|
|
|
try {
|
|
|
- rpcProxy.removeCachePool(null,
|
|
|
+ rpcProxy.removeCachePool(null,
|
|
|
RemoveCachePoolRequestProto.newBuilder().
|
|
|
- setPoolName(cachePoolName).build());
|
|
|
+ setPoolName(cachePoolName).build());
|
|
|
} catch (ServiceException e) {
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
private static class BatchedCachePoolEntries
|
|
|
- implements BatchedEntries<CachePoolEntry> {
|
|
|
- private final ListCachePoolsResponseProto proto;
|
|
|
-
|
|
|
+ implements BatchedEntries<CachePoolEntry> {
|
|
|
+ private final ListCachePoolsResponseProto proto;
|
|
|
+
|
|
|
public BatchedCachePoolEntries(ListCachePoolsResponseProto proto) {
|
|
|
this.proto = proto;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public CachePoolEntry get(int i) {
|
|
|
CachePoolEntryProto elem = proto.getEntries(i);
|
|
@@ -1253,7 +1208,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
public int size() {
|
|
|
return proto.getEntriesCount();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public boolean hasMore() {
|
|
|
return proto.getHasMore();
|
|
@@ -1265,9 +1220,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
throws IOException {
|
|
|
try {
|
|
|
return new BatchedCachePoolEntries(
|
|
|
- rpcProxy.listCachePools(null,
|
|
|
- ListCachePoolsRequestProto.newBuilder().
|
|
|
- setPrevPoolName(prevKey).build()));
|
|
|
+ rpcProxy.listCachePools(null,
|
|
|
+ ListCachePoolsRequestProto.newBuilder().
|
|
|
+ setPrevPoolName(prevKey).build()));
|
|
|
} catch (ServiceException e) {
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
|
}
|
|
@@ -1347,9 +1302,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
|
|
|
@Override
|
|
|
public void createEncryptionZone(String src, String keyName)
|
|
|
- throws IOException {
|
|
|
+ throws IOException {
|
|
|
final CreateEncryptionZoneRequestProto.Builder builder =
|
|
|
- CreateEncryptionZoneRequestProto.newBuilder();
|
|
|
+ CreateEncryptionZoneRequestProto.newBuilder();
|
|
|
builder.setSrc(src);
|
|
|
if (keyName != null && !keyName.isEmpty()) {
|
|
|
builder.setKeyName(keyName);
|
|
@@ -1363,8 +1318,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public EncryptionZone getEZForPath(String src)
|
|
|
- throws IOException {
|
|
|
+ public EncryptionZone getEZForPath(String src) throws IOException {
|
|
|
final GetEZForPathRequestProto.Builder builder =
|
|
|
GetEZForPathRequestProto.newBuilder();
|
|
|
builder.setSrc(src);
|
|
@@ -1386,9 +1340,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
public BatchedEntries<EncryptionZone> listEncryptionZones(long id)
|
|
|
throws IOException {
|
|
|
final ListEncryptionZonesRequestProto req =
|
|
|
- ListEncryptionZonesRequestProto.newBuilder()
|
|
|
- .setId(id)
|
|
|
- .build();
|
|
|
+ ListEncryptionZonesRequestProto.newBuilder()
|
|
|
+ .setId(id)
|
|
|
+ .build();
|
|
|
try {
|
|
|
EncryptionZonesProtos.ListEncryptionZonesResponseProto response =
|
|
|
rpcProxy.listEncryptionZones(null, req);
|
|
@@ -1397,8 +1351,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
for (EncryptionZoneProto p : response.getZonesList()) {
|
|
|
elements.add(PBHelperClient.convert(p));
|
|
|
}
|
|
|
- return new BatchedListEntries<EncryptionZone>(elements,
|
|
|
- response.getHasMore());
|
|
|
+ return new BatchedListEntries<>(elements, response.getHasMore());
|
|
|
} catch (ServiceException e) {
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
|
}
|
|
@@ -1418,7 +1371,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
|
|
|
throws IOException {
|
|
@@ -1434,11 +1387,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
- public List<XAttr> listXAttrs(String src)
|
|
|
- throws IOException {
|
|
|
- ListXAttrsRequestProto.Builder builder = ListXAttrsRequestProto.newBuilder();
|
|
|
+ public List<XAttr> listXAttrs(String src) throws IOException {
|
|
|
+ ListXAttrsRequestProto.Builder builder =
|
|
|
+ ListXAttrsRequestProto.newBuilder();
|
|
|
builder.setSrc(src);
|
|
|
ListXAttrsRequestProto req = builder.build();
|
|
|
try {
|