|
@@ -19,11 +19,16 @@ package org.apache.hadoop.hdfs.protocolPB;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.util.Arrays;
|
|
|
+import java.util.List;
|
|
|
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
|
+import org.apache.hadoop.fs.ContentSummary;
|
|
|
+import org.apache.hadoop.fs.FsServerDefaults;
|
|
|
import org.apache.hadoop.fs.Options.Rename;
|
|
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
|
+import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
@@ -124,9 +129,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CorruptFileBlocksProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto;
|
|
|
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
|
|
|
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.UpgradeStatusReportProto;
|
|
|
import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
|
|
|
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
|
|
|
import org.apache.hadoop.io.Text;
|
|
@@ -218,9 +221,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|
|
Builder builder = GetBlockLocationsResponseProto
|
|
|
.newBuilder();
|
|
|
if (b != null) {
|
|
|
- builder.setLocations(
|
|
|
- PBHelper.convert(server.getBlockLocations(req.getSrc(),
|
|
|
- req.getOffset(), req.getLength()))).build();
|
|
|
+ builder.setLocations(PBHelper.convert(b)).build();
|
|
|
}
|
|
|
return builder.build();
|
|
|
} catch (IOException e) {
|
|
@@ -233,14 +234,19 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|
|
RpcController controller, GetServerDefaultsRequestProto req)
|
|
|
throws ServiceException {
|
|
|
try {
|
|
|
+ FsServerDefaults result = server.getServerDefaults();
|
|
|
return GetServerDefaultsResponseProto.newBuilder()
|
|
|
- .setServerDefaults(PBHelper.convert(server.getServerDefaults()))
|
|
|
+ .setServerDefaults(PBHelper.convert(result))
|
|
|
.build();
|
|
|
} catch (IOException e) {
|
|
|
throw new ServiceException(e);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+
|
|
|
+ static final CreateResponseProto VOID_CREATE_RESPONSE =
|
|
|
+ CreateResponseProto.newBuilder().build();
|
|
|
+
|
|
|
@Override
|
|
|
public CreateResponseProto create(RpcController controller,
|
|
|
CreateRequestProto req) throws ServiceException {
|
|
@@ -252,19 +258,22 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|
|
} catch (IOException e) {
|
|
|
throw new ServiceException(e);
|
|
|
}
|
|
|
- return CreateResponseProto.newBuilder().build();
|
|
|
-
|
|
|
+ return VOID_CREATE_RESPONSE;
|
|
|
}
|
|
|
|
|
|
+ static final AppendResponseProto NULL_APPEND_RESPONSE =
|
|
|
+ AppendResponseProto.newBuilder().build();
|
|
|
+
|
|
|
@Override
|
|
|
public AppendResponseProto append(RpcController controller,
|
|
|
AppendRequestProto req) throws ServiceException {
|
|
|
try {
|
|
|
- return AppendResponseProto
|
|
|
- .newBuilder()
|
|
|
- .setBlock(
|
|
|
- PBHelper.convert(server.append(req.getSrc(), req.getClientName())))
|
|
|
- .build();
|
|
|
+ LocatedBlock result = server.append(req.getSrc(), req.getClientName());
|
|
|
+ if (result != null) {
|
|
|
+ return AppendResponseProto.newBuilder()
|
|
|
+ .setBlock(PBHelper.convert(result)).build();
|
|
|
+ }
|
|
|
+ return NULL_APPEND_RESPONSE;
|
|
|
} catch (IOException e) {
|
|
|
throw new ServiceException(e);
|
|
|
}
|
|
@@ -274,18 +283,16 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|
|
public SetReplicationResponseProto setReplication(RpcController controller,
|
|
|
SetReplicationRequestProto req) throws ServiceException {
|
|
|
try {
|
|
|
- return SetReplicationResponseProto
|
|
|
- .newBuilder()
|
|
|
- .setResult(
|
|
|
- server.setReplication(req.getSrc(), (short) req.getReplication()))
|
|
|
- .build();
|
|
|
+ boolean result =
|
|
|
+ server.setReplication(req.getSrc(), (short) req.getReplication());
|
|
|
+ return SetReplicationResponseProto.newBuilder().setResult(result).build();
|
|
|
} catch (IOException e) {
|
|
|
throw new ServiceException(e);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
|
|
|
- static final SetPermissionResponseProto SET_PERM_RESPONSE =
|
|
|
+ static final SetPermissionResponseProto VOID_SET_PERM_RESPONSE =
|
|
|
SetPermissionResponseProto.newBuilder().build();
|
|
|
|
|
|
@Override
|
|
@@ -296,24 +303,26 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|
|
} catch (IOException e) {
|
|
|
throw new ServiceException(e);
|
|
|
}
|
|
|
- return SET_PERM_RESPONSE;
|
|
|
+ return VOID_SET_PERM_RESPONSE;
|
|
|
}
|
|
|
|
|
|
- static final SetOwnerResponseProto SET_OWNER_RESPONSE =
|
|
|
+ static final SetOwnerResponseProto VOID_SET_OWNER_RESPONSE =
|
|
|
SetOwnerResponseProto.newBuilder().build();
|
|
|
|
|
|
@Override
|
|
|
public SetOwnerResponseProto setOwner(RpcController controller,
|
|
|
SetOwnerRequestProto req) throws ServiceException {
|
|
|
try {
|
|
|
- server.setOwner(req.getSrc(), req.getUsername(), req.getGroupname());
|
|
|
+ server.setOwner(req.getSrc(),
|
|
|
+ req.hasUsername() ? req.getUsername() : null,
|
|
|
+ req.hasGroupname() ? req.getGroupname() : null);
|
|
|
} catch (IOException e) {
|
|
|
throw new ServiceException(e);
|
|
|
}
|
|
|
- return SET_OWNER_RESPONSE;
|
|
|
+ return VOID_SET_OWNER_RESPONSE;
|
|
|
}
|
|
|
|
|
|
- static final AbandonBlockResponseProto ABD_BLOCK_RESPONSE =
|
|
|
+ static final AbandonBlockResponseProto VOID_ADD_BLOCK_RESPONSE =
|
|
|
AbandonBlockResponseProto.newBuilder().build();
|
|
|
|
|
|
@Override
|
|
@@ -325,20 +334,22 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|
|
} catch (IOException e) {
|
|
|
throw new ServiceException(e);
|
|
|
}
|
|
|
- return ABD_BLOCK_RESPONSE;
|
|
|
+ return VOID_ADD_BLOCK_RESPONSE;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public AddBlockResponseProto addBlock(RpcController controller,
|
|
|
AddBlockRequestProto req) throws ServiceException {
|
|
|
- try {
|
|
|
+
|
|
|
+ try {
|
|
|
+ List<DatanodeInfoProto> excl = req.getExcludeNodesList();
|
|
|
+ LocatedBlock result = server.addBlock(req.getSrc(), req.getClientName(),
|
|
|
+ req.hasPrevious() ? PBHelper.convert(req.getPrevious()) : null,
|
|
|
+ (excl == null ||
|
|
|
+ excl.size() == 0) ? null :
|
|
|
+ PBHelper.convert(excl.toArray(new DatanodeInfoProto[excl.size()])));
|
|
|
return AddBlockResponseProto.newBuilder().setBlock(
|
|
|
- PBHelper.convert(
|
|
|
- server.addBlock(req.getSrc(), req.getClientName(),
|
|
|
- req.hasPrevious() ? PBHelper.convert(req.getPrevious()) : null,
|
|
|
- PBHelper.convert(
|
|
|
- (DatanodeInfoProto[]) req.getExcludeNodesList().toArray()))))
|
|
|
- .build();
|
|
|
+ PBHelper.convert(result)).build();
|
|
|
} catch (IOException e) {
|
|
|
throw new ServiceException(e);
|
|
|
}
|
|
@@ -349,15 +360,17 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|
|
RpcController controller, GetAdditionalDatanodeRequestProto req)
|
|
|
throws ServiceException {
|
|
|
try {
|
|
|
+ List<DatanodeInfoProto> existingList = req.getExistingsList();
|
|
|
+ List<DatanodeInfoProto> excludesList = req.getExcludesList();
|
|
|
+ LocatedBlock result = server.getAdditionalDatanode(
|
|
|
+ req.getSrc(), PBHelper.convert(req.getBlk()),
|
|
|
+ PBHelper.convert(existingList.toArray(
|
|
|
+ new DatanodeInfoProto[existingList.size()])),
|
|
|
+ PBHelper.convert(excludesList.toArray(
|
|
|
+ new DatanodeInfoProto[excludesList.size()])),
|
|
|
+ req.getNumAdditionalNodes(), req.getClientName());
|
|
|
return GetAdditionalDatanodeResponseProto.newBuilder().setBlock(
|
|
|
- PBHelper.convert(
|
|
|
- server.getAdditionalDatanode(req.getSrc(),
|
|
|
- PBHelper.convert(req.getBlk()),
|
|
|
- PBHelper.convert((DatanodeInfoProto[]) req.getExistingsList()
|
|
|
- .toArray()), PBHelper
|
|
|
- .convert((DatanodeInfoProto[]) req.getExcludesList()
|
|
|
- .toArray()), req.getNumAdditionalNodes(), req
|
|
|
- .getClientName())))
|
|
|
+ PBHelper.convert(result))
|
|
|
.build();
|
|
|
} catch (IOException e) {
|
|
|
throw new ServiceException(e);
|
|
@@ -368,10 +381,10 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|
|
public CompleteResponseProto complete(RpcController controller,
|
|
|
CompleteRequestProto req) throws ServiceException {
|
|
|
try {
|
|
|
- return CompleteResponseProto.newBuilder().setResult(
|
|
|
- server.complete(req.getSrc(), req.getClientName(),
|
|
|
- PBHelper.convert(req.getLast())))
|
|
|
- .build();
|
|
|
+ boolean result =
|
|
|
+ server.complete(req.getSrc(), req.getClientName(),
|
|
|
+ req.hasLast() ? PBHelper.convert(req.getLast()) : null);
|
|
|
+ return CompleteResponseProto.newBuilder().setResult(result).build();
|
|
|
} catch (IOException e) {
|
|
|
throw new ServiceException(e);
|
|
|
}
|
|
@@ -384,8 +397,9 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|
|
public ReportBadBlocksResponseProto reportBadBlocks(RpcController controller,
|
|
|
ReportBadBlocksRequestProto req) throws ServiceException {
|
|
|
try {
|
|
|
+ List<LocatedBlockProto> bl = req.getBlocksList();
|
|
|
server.reportBadBlocks(PBHelper.convertLocatedBlock(
|
|
|
- (LocatedBlockProto[]) req.getBlocksList().toArray()));
|
|
|
+ bl.toArray(new LocatedBlockProto[bl.size()])));
|
|
|
} catch (IOException e) {
|
|
|
throw new ServiceException(e);
|
|
|
}
|
|
@@ -399,7 +413,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|
|
public ConcatResponseProto concat(RpcController controller,
|
|
|
ConcatRequestProto req) throws ServiceException {
|
|
|
try {
|
|
|
- server.concat(req.getTrg(), (String[])req.getSrcsList().toArray());
|
|
|
+ List<String> srcs = req.getSrcsList();
|
|
|
+ server.concat(req.getTrg(), srcs.toArray(new String[srcs.size()]));
|
|
|
} catch (IOException e) {
|
|
|
throw new ServiceException(e);
|
|
|
}
|
|
@@ -456,14 +471,21 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ static final GetListingResponseProto NULL_GETLISTING_RESPONSE =
|
|
|
+ GetListingResponseProto.newBuilder().build();
|
|
|
@Override
|
|
|
public GetListingResponseProto getListing(RpcController controller,
|
|
|
GetListingRequestProto req) throws ServiceException {
|
|
|
try {
|
|
|
- DirectoryListingProto result = PBHelper.convert(server.getListing(
|
|
|
+ DirectoryListing result = server.getListing(
|
|
|
req.getSrc(), req.getStartAfter().toByteArray(),
|
|
|
- req.getNeedLocation()));
|
|
|
- return GetListingResponseProto.newBuilder().setDirList(result).build();
|
|
|
+ req.getNeedLocation());
|
|
|
+ if (result !=null) {
|
|
|
+ return GetListingResponseProto.newBuilder().setDirList(
|
|
|
+ PBHelper.convert(result)).build();
|
|
|
+ } else {
|
|
|
+ return NULL_GETLISTING_RESPONSE;
|
|
|
+ }
|
|
|
} catch (IOException e) {
|
|
|
throw new ServiceException(e);
|
|
|
}
|
|
@@ -494,6 +516,19 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ public RestoreFailedStorageResponseProto restoreFailedStorage(
|
|
|
+ RpcController controller, RestoreFailedStorageRequestProto req)
|
|
|
+ throws ServiceException {
|
|
|
+ try {
|
|
|
+ boolean result = server.restoreFailedStorage(req.getArg());
|
|
|
+ return RestoreFailedStorageResponseProto.newBuilder().setResult(result)
|
|
|
+ .build();
|
|
|
+ } catch (IOException e) {
|
|
|
+ throw new ServiceException(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public GetFsStatsResponseProto getFsStats(RpcController controller,
|
|
|
GetFsStatusRequestProto req) throws ServiceException {
|
|
@@ -557,19 +592,6 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|
|
|
|
|
}
|
|
|
|
|
|
- @Override
|
|
|
- public RestoreFailedStorageResponseProto restoreFailedStorage(
|
|
|
- RpcController controller, RestoreFailedStorageRequestProto req)
|
|
|
- throws ServiceException {
|
|
|
- try {
|
|
|
- boolean result = server.restoreFailedStorage(req.getArg());
|
|
|
- return RestoreFailedStorageResponseProto.newBuilder().setResult(result)
|
|
|
- .build();
|
|
|
- } catch (IOException e) {
|
|
|
- throw new ServiceException(e);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
static final RefreshNodesResponseProto VOID_REFRESHNODES_RESPONSE =
|
|
|
RefreshNodesResponseProto.newBuilder().build();
|
|
|
|
|
@@ -622,9 +644,10 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|
|
RpcController controller, ListCorruptFileBlocksRequestProto req)
|
|
|
throws ServiceException {
|
|
|
try {
|
|
|
- CorruptFileBlocksProto result = PBHelper.convert(server
|
|
|
- .listCorruptFileBlocks(req.getPath(), req.getCookie()));
|
|
|
- return ListCorruptFileBlocksResponseProto.newBuilder().setCorrupt(result)
|
|
|
+ CorruptFileBlocks result = server.listCorruptFileBlocks(
|
|
|
+ req.getPath(), req.hasCookie() ? req.getCookie(): null);
|
|
|
+ return ListCorruptFileBlocksResponseProto.newBuilder()
|
|
|
+ .setCorrupt(PBHelper.convert(result))
|
|
|
.build();
|
|
|
} catch (IOException e) {
|
|
|
throw new ServiceException(e);
|
|
@@ -646,29 +669,40 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|
|
|
|
|
}
|
|
|
|
|
|
+ static final GetFileInfoResponseProto NULL_GETFILEINFO_RESPONSE =
|
|
|
+ GetFileInfoResponseProto.newBuilder().build();
|
|
|
@Override
|
|
|
public GetFileInfoResponseProto getFileInfo(RpcController controller,
|
|
|
GetFileInfoRequestProto req) throws ServiceException {
|
|
|
try {
|
|
|
- HdfsFileStatus res = server.getFileInfo(req.getSrc());
|
|
|
- GetFileInfoResponseProto.Builder builder =
|
|
|
- GetFileInfoResponseProto.newBuilder();
|
|
|
- if (res != null) {
|
|
|
- builder.setFs(PBHelper.convert(res));
|
|
|
+ HdfsFileStatus result = server.getFileInfo(req.getSrc());
|
|
|
+
|
|
|
+ if (result != null) {
|
|
|
+ return GetFileInfoResponseProto.newBuilder().setFs(
|
|
|
+ PBHelper.convert(result)).build();
|
|
|
}
|
|
|
- return builder.build();
|
|
|
+ return NULL_GETFILEINFO_RESPONSE;
|
|
|
} catch (IOException e) {
|
|
|
throw new ServiceException(e);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ static final GetFileLinkInfoResponseProto NULL_GETFILELINKINFO_RESPONSE =
|
|
|
+ GetFileLinkInfoResponseProto.newBuilder().build();
|
|
|
@Override
|
|
|
public GetFileLinkInfoResponseProto getFileLinkInfo(RpcController controller,
|
|
|
GetFileLinkInfoRequestProto req) throws ServiceException {
|
|
|
try {
|
|
|
- HdfsFileStatusProto result =
|
|
|
- PBHelper.convert(server.getFileLinkInfo(req.getSrc()));
|
|
|
- return GetFileLinkInfoResponseProto.newBuilder().setFs(result).build();
|
|
|
+ HdfsFileStatus result = server.getFileLinkInfo(req.getSrc());
|
|
|
+ if (result != null) {
|
|
|
+ System.out.println("got non null result for getFileLinkInfo for " + req.getSrc());
|
|
|
+ return GetFileLinkInfoResponseProto.newBuilder().setFs(
|
|
|
+ PBHelper.convert(result)).build();
|
|
|
+ } else {
|
|
|
+ System.out.println("got null result for getFileLinkInfo for " + req.getSrc());
|
|
|
+ return NULL_GETFILELINKINFO_RESPONSE;
|
|
|
+ }
|
|
|
+
|
|
|
} catch (IOException e) {
|
|
|
throw new ServiceException(e);
|
|
|
}
|
|
@@ -679,10 +713,9 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|
|
RpcController controller, GetContentSummaryRequestProto req)
|
|
|
throws ServiceException {
|
|
|
try {
|
|
|
- ContentSummaryProto result =
|
|
|
- PBHelper.convert(server.getContentSummary(req.getPath()));
|
|
|
- return
|
|
|
- GetContentSummaryResponseProto.newBuilder().setSummary(result).build();
|
|
|
+ ContentSummary result = server.getContentSummary(req.getPath());
|
|
|
+ return GetContentSummaryResponseProto.newBuilder()
|
|
|
+ .setSummary(PBHelper.convert(result)).build();
|
|
|
} catch (IOException e) {
|
|
|
throw new ServiceException(e);
|
|
|
}
|
|
@@ -780,10 +813,11 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|
|
public UpdatePipelineResponseProto updatePipeline(RpcController controller,
|
|
|
UpdatePipelineRequestProto req) throws ServiceException {
|
|
|
try {
|
|
|
+ List<DatanodeIDProto> newNodes = req.getNewNodesList();
|
|
|
server
|
|
|
.updatePipeline(req.getClientName(), PBHelper.convert(req
|
|
|
.getOldBlock()), PBHelper.convert(req.getNewBlock()), PBHelper
|
|
|
- .convert((DatanodeIDProto[]) req.getNewNodesList().toArray()));
|
|
|
+ .convert(newNodes.toArray(new DatanodeIDProto[newNodes.size()])));
|
|
|
return VOID_UPDATEPIPELINE_RESPONSE;
|
|
|
} catch (IOException e) {
|
|
|
throw new ServiceException(e);
|