|
@@ -20,7 +20,10 @@ package org.apache.hadoop.hdfs.protocolPB;
|
|
|
import java.io.Closeable;
|
|
|
import java.io.FileNotFoundException;
|
|
|
import java.io.IOException;
|
|
|
+import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
+import java.util.List;
|
|
|
+import java.util.NoSuchElementException;
|
|
|
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
@@ -30,9 +33,19 @@ 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.RemoteIterator;
|
|
|
import org.apache.hadoop.fs.UnresolvedLinkException;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
|
|
|
+import org.apache.hadoop.hdfs.protocol.PathCacheDirective;
|
|
|
+import org.apache.hadoop.hdfs.protocol.PathCacheEntry;
|
|
|
+import org.apache.hadoop.hdfs.protocol.AddPathCacheDirectiveException.EmptyPathError;
|
|
|
+import org.apache.hadoop.hdfs.protocol.AddPathCacheDirectiveException.InvalidPathNameError;
|
|
|
+import org.apache.hadoop.hdfs.protocol.AddPathCacheDirectiveException.InvalidPoolNameError;
|
|
|
+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.UnexpectedRemovePathCacheEntryException;
|
|
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
|
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
|
|
|
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
|
|
@@ -50,6 +63,10 @@ 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.PathCacheDirectiveProto;
|
|
|
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddPathCacheDirectiveErrorProto;
|
|
|
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddPathCacheDirectivesRequestProto;
|
|
|
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddPathCacheDirectivesResponseProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
|
|
@@ -87,11 +104,18 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSna
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
|
|
|
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListPathCacheEntriesElementProto;
|
|
|
+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.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.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.Rename2RequestProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
|
|
|
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto;
|
|
@@ -127,6 +151,7 @@ import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequest
|
|
|
import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProto;
|
|
|
import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
+import org.apache.hadoop.util.Fallible;
|
|
|
|
|
|
import com.google.protobuf.ByteString;
|
|
|
import com.google.protobuf.ServiceException;
|
|
@@ -982,4 +1007,170 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|
|
throw ProtobufHelper.getRemoteException(e);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ private static IOException addPathCacheDirectivesError(long code,
|
|
|
+ PathCacheDirective directive) {
|
|
|
+ if (code == AddPathCacheDirectiveErrorProto.EMPTY_PATH_ERROR_VALUE) {
|
|
|
+ return new EmptyPathError(directive);
|
|
|
+ } else if (code == AddPathCacheDirectiveErrorProto.
|
|
|
+ INVALID_PATH_NAME_ERROR_VALUE) {
|
|
|
+ return new InvalidPathNameError(directive);
|
|
|
+ } else if (code == AddPathCacheDirectiveErrorProto.
|
|
|
+ INVALID_POOL_NAME_ERROR_VALUE) {
|
|
|
+ return new InvalidPoolNameError(directive);
|
|
|
+ } else {
|
|
|
+ return new UnexpectedAddPathCacheDirectiveException(directive);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public List<Fallible<PathCacheEntry>> addPathCacheDirectives(
|
|
|
+ List<PathCacheDirective> directives) throws IOException {
|
|
|
+ try {
|
|
|
+ AddPathCacheDirectivesRequestProto.Builder builder =
|
|
|
+ AddPathCacheDirectivesRequestProto.newBuilder();
|
|
|
+ for (PathCacheDirective directive : directives) {
|
|
|
+ builder.addElements(PathCacheDirectiveProto.newBuilder().
|
|
|
+ setPath(directive.getPath()).
|
|
|
+ setPool(directive.getPool()).
|
|
|
+ build());
|
|
|
+ }
|
|
|
+ AddPathCacheDirectivesResponseProto result =
|
|
|
+ rpcProxy.addPathCacheDirectives(null, builder.build());
|
|
|
+ int resultsCount = result.getResultsCount();
|
|
|
+ ArrayList<Fallible<PathCacheEntry>> results =
|
|
|
+ new ArrayList<Fallible<PathCacheEntry>>(resultsCount);
|
|
|
+ for (int i = 0; i < resultsCount; i++) {
|
|
|
+ PathCacheDirective directive = directives.get(i);
|
|
|
+ long code = result.getResults(i);
|
|
|
+ if (code > 0) {
|
|
|
+ results.add(new Fallible<PathCacheEntry>(
|
|
|
+ new PathCacheEntry(code, directive)));
|
|
|
+ } else {
|
|
|
+ results.add(new Fallible<PathCacheEntry>(
|
|
|
+ addPathCacheDirectivesError(code, directive)));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return results;
|
|
|
+ } catch (ServiceException e) {
|
|
|
+ throw ProtobufHelper.getRemoteException(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private static IOException removePathCacheEntriesError(long code, long id) {
|
|
|
+ if (code == RemovePathCacheEntryErrorProto.
|
|
|
+ INVALID_CACHED_PATH_ID_ERROR_VALUE) {
|
|
|
+ return new InvalidIdException(id);
|
|
|
+ } else if (code == RemovePathCacheEntryErrorProto.
|
|
|
+ NO_SUCH_CACHED_PATH_ID_ERROR_VALUE) {
|
|
|
+ return new NoSuchIdException(id);
|
|
|
+ } else {
|
|
|
+ return new UnexpectedRemovePathCacheEntryException(id);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public List<Fallible<Long>> removePathCacheEntries(List<Long> ids)
|
|
|
+ throws IOException {
|
|
|
+ try {
|
|
|
+ RemovePathCacheEntriesRequestProto.Builder builder =
|
|
|
+ RemovePathCacheEntriesRequestProto.newBuilder();
|
|
|
+ for (Long id : ids) {
|
|
|
+ builder.addElements(id);
|
|
|
+ }
|
|
|
+ RemovePathCacheEntriesResponseProto result =
|
|
|
+ rpcProxy.removePathCacheEntries(null, builder.build());
|
|
|
+ int resultsCount = result.getResultsCount();
|
|
|
+ ArrayList<Fallible<Long>> results =
|
|
|
+ new ArrayList<Fallible<Long>>(resultsCount);
|
|
|
+ for (int i = 0; i < resultsCount; i++) {
|
|
|
+ long code = result.getResults(i);
|
|
|
+ if (code > 0) {
|
|
|
+ results.add(new Fallible<Long>(code));
|
|
|
+ } else {
|
|
|
+ results.add(new Fallible<Long>(
|
|
|
+ removePathCacheEntriesError(code, ids.get(i))));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return results;
|
|
|
+ } catch (ServiceException e) {
|
|
|
+ throw ProtobufHelper.getRemoteException(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private class PathCacheEntriesIterator
|
|
|
+ implements RemoteIterator<PathCacheEntry> {
|
|
|
+ private long prevId;
|
|
|
+ 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;
|
|
|
+ this.pool = pool;
|
|
|
+ this.repliesPerRequest = repliesPerRequest;
|
|
|
+ this.response = null;
|
|
|
+ this.idx = -1;
|
|
|
+ }
|
|
|
+
|
|
|
+ private void makeRequest() throws IOException {
|
|
|
+ idx = 0;
|
|
|
+ response = null;
|
|
|
+ try {
|
|
|
+ ListPathCacheEntriesRequestProto req =
|
|
|
+ ListPathCacheEntriesRequestProto.newBuilder().
|
|
|
+ setPrevId(prevId).
|
|
|
+ setPool(pool).
|
|
|
+ setMaxReplies(repliesPerRequest).
|
|
|
+ build();
|
|
|
+ response = rpcProxy.listPathCacheEntries(null, req);
|
|
|
+ if (response.getElementsCount() == 0) {
|
|
|
+ response = null;
|
|
|
+ }
|
|
|
+ } catch (ServiceException e) {
|
|
|
+ throw ProtobufHelper.getRemoteException(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ 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 boolean hasNext() throws IOException {
|
|
|
+ makeRequestIfNeeded();
|
|
|
+ return (response != null);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public PathCacheEntry next() throws IOException {
|
|
|
+ makeRequestIfNeeded();
|
|
|
+ if (response == null) {
|
|
|
+ throw new NoSuchElementException();
|
|
|
+ }
|
|
|
+ ListPathCacheEntriesElementProto elementProto =
|
|
|
+ response.getElements(idx);
|
|
|
+ prevId = elementProto.getId();
|
|
|
+ idx++;
|
|
|
+ return new PathCacheEntry(elementProto.getId(),
|
|
|
+ new PathCacheDirective(elementProto.getPath(),
|
|
|
+ elementProto.getPool()));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public RemoteIterator<PathCacheEntry> listPathCacheEntries(long prevId,
|
|
|
+ String pool, int repliesPerRequest) throws IOException {
|
|
|
+ return new PathCacheEntriesIterator(prevId, pool, repliesPerRequest);
|
|
|
+ }
|
|
|
}
|