HDS-2895. Remove Writable wire protocol types and translators to complete transition to protocol buffers. Contributed by Suresh Srinivas.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1241007 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Suresh Srinivas 2012-02-06 13:33:35 +00:00
parent b837bbb7d5
commit 2a9e430ff9
66 changed files with 35 additions and 8304 deletions

View File

@ -119,6 +119,9 @@ Trunk (unreleased changes)
HDFS-2785. Update webhdfs and httpfs for host-based token support.
(Robert Joseph Evans via jitendra)
HDS-2895. Remove Writable wire protocol types and translators to
complete transition to protocol buffers. (suresh)
OPTIMIZATIONS
HDFS-2477. Optimize computing the diff between a block report and the
namenode state. (Tomasz Nykiel via hairong)

View File

@ -49,10 +49,11 @@ public interface ClientDatanodeProtocol extends VersionedProtocol {
* DN server side to insulate from the protocol serialization.
*
* If you are adding/changing DN's interface then you need to
* change both this class and ALSO
* {@link org.apache.hadoop.hdfs.protocolR23Compatible.ClientDatanodeWireProtocol}.
* These changes need to be done in a compatible fashion as described in
* {@link org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol}
* change both this class and ALSO related protocol buffer
* wire protocol definition in ClientDatanodeProtocol.proto.
*
* For more details on protocol buffer wire protocol, please see
* .../org/apache/hadoop/hdfs/protocolPB/overview.html
*
* The log of historical changes can be retrieved from the svn).
* 9: Added deleteBlockPool method

View File

@ -72,11 +72,12 @@ public interface ClientProtocol extends VersionedProtocol {
* This class is used by both the DFSClient and the
* NN server side to insulate from the protocol serialization.
*
* If you are adding/changing NN's interface then you need to
* change both this class and ALSO
* {@link org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol}.
* These changes need to be done in a compatible fashion as described in
* {@link org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol}
* If you are adding/changing this interface then you need to
* change both this class and ALSO related protocol buffer
* wire protocol definition in ClientNamenodeProtocol.proto.
*
* For more details on protocol buffer wire protocol, please see
* .../org/apache/hadoop/hdfs/protocolPB/overview.html
*
* The log of historical changes can be retrieved from the svn).
* 69: Eliminate overloaded method names.

View File

@ -127,8 +127,6 @@
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTimesRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto;
import com.google.protobuf.ByteString;
import com.google.protobuf.ServiceException;

View File

@ -58,7 +58,6 @@
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
import org.apache.hadoop.hdfs.server.protocolR23Compatible.DatanodeWireProtocol;
import org.apache.hadoop.io.retry.RetryPolicies;
import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.io.retry.RetryProxy;
@ -105,7 +104,7 @@ private static DatanodeProtocolPB createNamenode(
UserGroupInformation ugi) throws IOException {
return RPC.getProxy(DatanodeProtocolPB.class,
RPC.getProtocolVersion(DatanodeProtocolPB.class), nameNodeAddr, ugi,
conf, NetUtils.getSocketFactory(conf, DatanodeWireProtocol.class));
conf, NetUtils.getSocketFactory(conf, DatanodeProtocolPB.class));
}
/** Create a {@link NameNode} proxy */

View File

@ -1,111 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
/**************************************************
* A Block is a Hadoop FS primitive, identified by a long.
**************************************************/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class BlockWritable implements Writable {
static { // register a ctor
WritableFactories.setFactory
(BlockWritable.class,
new WritableFactory() {
public Writable newInstance() { return new BlockWritable(); }
});
}
private long blockId;
private long numBytes;
private long generationStamp;
public BlockWritable() {this(0, 0, 0);}
public BlockWritable(final long blkid, final long len, final long genStamp) {
this.blockId = blkid;
this.numBytes = len;
this.generationStamp = genStamp;
}
/////////////////////////////////////
// Writable
/////////////////////////////////////
@Override // Writable
public void write(DataOutput out) throws IOException {
out.writeLong(blockId);
out.writeLong(numBytes);
out.writeLong(generationStamp);
}
@Override // Writable
public void readFields(DataInput in) throws IOException {
this.blockId = in.readLong();
this.numBytes = in.readLong();
this.generationStamp = in.readLong();
}
public static BlockWritable convert(Block b) {
return new BlockWritable(b.getBlockId(), b.getNumBytes(),
b.getGenerationStamp());
}
public Block convert() {
return new Block(blockId, numBytes, generationStamp);
}
public long getBlockId() {
return blockId;
}
public long getNumBytes() {
return numBytes;
}
public long getGenerationStamp() {
return generationStamp;
}
public static Block[] convert(BlockWritable[] blocks) {
Block[] ret = new Block[blocks.length];
for (int i = 0; i < blocks.length; i++) {
ret[i] = blocks[i].convert();
}
return ret;
}
public static BlockWritable[] convert(Block[] blocks) {
BlockWritable[] ret = new BlockWritable[blocks.length];
for (int i = 0; i < blocks.length; i++) {
ret[i] = BlockWritable.convert(blocks[i]);
}
return ret;
}
}

View File

@ -1,129 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
/** A class to implement an array of BlockLocations
* It provide efficient customized serialization/deserialization methods
* in stead of using the default array (de)serialization provided by RPC
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class BlocksWithLocationsWritable implements Writable {
/**
* A class to keep track of a block and its locations
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public static class BlockWithLocationsWritable implements Writable {
private BlockWritable block;
private String datanodeIDs[];
/** default constructor */
public BlockWithLocationsWritable() {
block = new BlockWritable();
datanodeIDs = null;
}
/** constructor */
public BlockWithLocationsWritable(BlockWritable b, String[] datanodes) {
block = b;
datanodeIDs = datanodes;
}
/** deserialization method */
public void readFields(DataInput in) throws IOException {
block.readFields(in);
int len = WritableUtils.readVInt(in); // variable length integer
datanodeIDs = new String[len];
for(int i=0; i<len; i++) {
datanodeIDs[i] = Text.readString(in);
}
}
/** serialization method */
public void write(DataOutput out) throws IOException {
block.write(out);
WritableUtils.writeVInt(out, datanodeIDs.length); // variable length int
for(String id:datanodeIDs) {
Text.writeString(out, id);
}
}
}
private BlockWithLocationsWritable[] blocks;
/** default constructor */
BlocksWithLocationsWritable() {
}
/** Constructor with one parameter */
public BlocksWithLocationsWritable( BlockWithLocationsWritable[] blocks ) {
this.blocks = blocks;
}
/** serialization method */
public void write( DataOutput out ) throws IOException {
WritableUtils.writeVInt(out, blocks.length);
for(int i=0; i<blocks.length; i++) {
blocks[i].write(out);
}
}
/** deserialization method */
public void readFields(DataInput in) throws IOException {
int len = WritableUtils.readVInt(in);
blocks = new BlockWithLocationsWritable[len];
for(int i=0; i<len; i++) {
blocks[i] = new BlockWithLocationsWritable();
blocks[i].readFields(in);
}
}
public static BlocksWithLocationsWritable convert(BlocksWithLocations locs) {
BlockWithLocations[] blocks = locs.getBlocks();
BlockWithLocationsWritable[] blocksWritable =
new BlockWithLocationsWritable[blocks.length];
for (int i = 0; i < blocks.length; i++) {
blocksWritable[i] = new BlockWithLocationsWritable(
BlockWritable.convert(blocks[i].getBlock()), blocks[i].getDatanodes());
}
return new BlocksWithLocationsWritable(blocksWritable);
}
public BlocksWithLocations convert() {
BlockWithLocations[] locs = new BlockWithLocations[blocks.length];
for (int i = 0; i < blocks.length; i++) {
locs[i] = new BlockWithLocations(blocks[i].block.convert(),
blocks[i].datanodeIDs);
}
return new BlocksWithLocations(locs);
}
}

View File

@ -1,92 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
import org.apache.hadoop.io.WritableUtils;
/**
* A unique signature intended to identify checkpoint transactions.
*/
@InterfaceAudience.Private
public class CheckpointSignatureWritable implements Writable {
private String blockpoolID = "";
private long mostRecentCheckpointTxId;
private long curSegmentTxId;
private StorageInfoWritable storageInfo;
public CheckpointSignatureWritable() {}
CheckpointSignatureWritable(long mostRecentCheckpointTxId,
long curSegmentTxId, int layoutVersion, int namespaceID, String bpid,
String clusterID, long cTime) {
this.blockpoolID = bpid;
this.mostRecentCheckpointTxId = mostRecentCheckpointTxId;
this.curSegmentTxId = curSegmentTxId;
this.storageInfo = new StorageInfoWritable(layoutVersion, namespaceID,
clusterID, cTime);
}
/////////////////////////////////////////////////
// Writable
/////////////////////////////////////////////////
static {
WritableFactories.setFactory(CheckpointSignatureWritable.class,
new WritableFactory() {
public Writable newInstance() {
return new CheckpointSignatureWritable();
}
});
}
@Override
public void write(DataOutput out) throws IOException {
storageInfo.write(out);
WritableUtils.writeString(out, blockpoolID);
out.writeLong(mostRecentCheckpointTxId);
out.writeLong(curSegmentTxId);
}
@Override
public void readFields(DataInput in) throws IOException {
storageInfo.readFields(in);
blockpoolID = WritableUtils.readString(in);
mostRecentCheckpointTxId = in.readLong();
curSegmentTxId = in.readLong();
}
public static CheckpointSignatureWritable convert(
CheckpointSignature sig) {
return new CheckpointSignatureWritable(sig.getMostRecentCheckpointTxId(),
sig.getCurSegmentTxId(), sig.getLayoutVersion(), sig.getNamespaceID(),
sig.getBlockpoolID(), sig.getClusterID(), sig.getCTime());
}
public CheckpointSignature convert() {
return new CheckpointSignature(storageInfo.convert(), blockpoolID,
mostRecentCheckpointTxId, curSegmentTxId);
}
}

View File

@ -1,129 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.security.token.Token;
/**
* This class is used on the server side.
* Calls come across the wire for the protocol family of Release 23 onwards.
* This class translates the R23 data types to the internal data types used
* inside the DN as specified in the generic ClientDatanodeProtocol.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class ClientDatanodeProtocolServerSideTranslatorR23 implements
ClientDatanodeWireProtocol {
final private ClientDatanodeProtocol server;
/**
*
* @param server - the NN server
* @throws IOException
*/
public ClientDatanodeProtocolServerSideTranslatorR23(
ClientDatanodeProtocol server) throws IOException {
this.server = server;
}
/**
* the client side will redirect getProtocolSignature to
* getProtocolSignature2.
*
* However the RPC layer below on the Server side will call
* getProtocolVersion and possibly in the future getProtocolSignature.
* Hence we still implement it even though the end client's call will
* never reach here.
*/
@Override
public ProtocolSignature getProtocolSignature(String protocol,
long clientVersion, int clientMethodsHash) throws IOException {
/**
* Don't forward this to the server. The protocol version and
* signature is that of {@link ClientDatanodeProtocol}
*/
if (!protocol.equals(RPC.getProtocolName(
ClientDatanodeWireProtocol.class))) {
throw new IOException("Datanode Serverside implements " +
ClientDatanodeWireProtocol.class +
". The following requested protocol is unknown: " + protocol);
}
return ProtocolSignature.getProtocolSignature(clientMethodsHash,
ClientDatanodeWireProtocol.versionID,
ClientDatanodeWireProtocol.class);
}
@Override
public ProtocolSignatureWritable
getProtocolSignature2(
String protocol, long clientVersion, int clientMethodsHash)
throws IOException {
/**
* Don't forward this to the server. The protocol version and
* signature is that of {@link ClientNamenodeProtocol}
*/
return ProtocolSignatureWritable.convert(
this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
}
@Override
public long getProtocolVersion(String protocol, long clientVersion) throws IOException {
if (protocol.equals(RPC.getProtocolName(
ClientDatanodeWireProtocol.class))) {
return ClientDatanodeWireProtocol.versionID;
}
throw new IOException("Datanode Serverside implements " +
ClientDatanodeWireProtocol.class +
". The following requested protocol is unknown: " + protocol);
}
@Override
public long getReplicaVisibleLength(ExtendedBlockWritable b) throws IOException {
return
server.getReplicaVisibleLength(ExtendedBlockWritable.convertExtendedBlock(b));
}
@Override
public void refreshNamenodes() throws IOException {
server.refreshNamenodes();
}
@Override
public void deleteBlockPool(String bpid, boolean force) throws IOException {
server.deleteBlockPool(bpid, force);
}
@Override
public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
Token<BlockTokenIdentifier> token) throws IOException {
return server.getBlockLocalPathInfo(block, token);
}
}

View File

@ -1,162 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.IOException;
import java.net.InetSocketAddress;
import javax.net.SocketFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
/**
* This class forwards ClientDatanodeProtocol calls as RPC to the DN server
* while translating from the parameter types used in ClientDatanodeProtocol to
* those used in protocolR23Compatile.*.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class ClientDatanodeProtocolTranslatorR23 implements
ClientDatanodeProtocol {
final private ClientDatanodeWireProtocol rpcProxy;
public ClientDatanodeProtocolTranslatorR23(DatanodeID datanodeid,
Configuration conf, int socketTimeout, LocatedBlock locatedBlock)
throws IOException {
rpcProxy = createClientDatanodeProtocolProxy( datanodeid, conf,
socketTimeout, locatedBlock);
}
/** used for testing */
public ClientDatanodeProtocolTranslatorR23(InetSocketAddress addr,
UserGroupInformation ticket,
Configuration conf,
SocketFactory factory) throws IOException {
rpcProxy = createClientDatanodeProtocolProxy(addr, ticket, conf, factory);
}
/**
* Constructor.
* @param datanodeid Datanode to connect to.
* @param conf Configuration.
* @param socketTimeout Socket timeout to use.
* @throws IOException
*/
public ClientDatanodeProtocolTranslatorR23(DatanodeID datanodeid,
Configuration conf, int socketTimeout) throws IOException {
InetSocketAddress addr = NetUtils.createSocketAddr(datanodeid.getHost()
+ ":" + datanodeid.getIpcPort());
rpcProxy = RPC.getProxy(ClientDatanodeWireProtocol.class,
ClientDatanodeWireProtocol.versionID, addr,
UserGroupInformation.getCurrentUser(), conf,
NetUtils.getDefaultSocketFactory(conf), socketTimeout);
}
static ClientDatanodeWireProtocol createClientDatanodeProtocolProxy(
DatanodeID datanodeid, Configuration conf, int socketTimeout,
LocatedBlock locatedBlock)
throws IOException {
InetSocketAddress addr = NetUtils.createSocketAddr(
datanodeid.getHost() + ":" + datanodeid.getIpcPort());
if (ClientDatanodeWireProtocol.LOG.isDebugEnabled()) {
ClientDatanodeWireProtocol.LOG.debug(
"ClientDatanodeProtocol addr=" + addr);
}
// Since we're creating a new UserGroupInformation here, we know that no
// future RPC proxies will be able to re-use the same connection. And
// usages of this proxy tend to be one-off calls.
//
// This is a temporary fix: callers should really achieve this by using
// RPC.stopProxy() on the resulting object, but this is currently not
// working in trunk. See the discussion on HDFS-1965.
Configuration confWithNoIpcIdle = new Configuration(conf);
confWithNoIpcIdle.setInt(CommonConfigurationKeysPublic
.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, 0);
UserGroupInformation ticket = UserGroupInformation
.createRemoteUser(locatedBlock.getBlock().getLocalBlock().toString());
ticket.addToken(locatedBlock.getBlockToken());
return RPC.getProxy(ClientDatanodeWireProtocol.class,
ClientDatanodeWireProtocol.versionID, addr, ticket, confWithNoIpcIdle,
NetUtils.getDefaultSocketFactory(conf), socketTimeout);
}
static ClientDatanodeWireProtocol createClientDatanodeProtocolProxy(
InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
SocketFactory factory) throws IOException {
return RPC.getProxy(ClientDatanodeWireProtocol.class,
ClientDatanodeWireProtocol.versionID, addr, ticket, conf,
factory);
}
@Override
public ProtocolSignature getProtocolSignature(
String protocolName, long clientVersion, int clientMethodHash)
throws IOException {
return ProtocolSignatureWritable.convert(
rpcProxy.getProtocolSignature2(
protocolName, clientVersion, clientMethodHash));
}
@Override
public long getProtocolVersion(String protocolName, long clientVersion)
throws IOException {
return rpcProxy.getProtocolVersion(protocolName, clientVersion);
}
@Override
public long getReplicaVisibleLength(ExtendedBlock b) throws IOException {
return rpcProxy.getReplicaVisibleLength(
ExtendedBlockWritable.convertExtendedBlock(b));
}
@Override
public void refreshNamenodes() throws IOException {
rpcProxy.refreshNamenodes();
}
@Override
public void deleteBlockPool(String bpid, boolean force) throws IOException {
rpcProxy.deleteBlockPool(bpid, force);
}
@Override
public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
Token<BlockTokenIdentifier> token) throws IOException {
return rpcProxy.getBlockLocalPathInfo(block, token);
}
}

View File

@ -1,100 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSelector;
import org.apache.hadoop.ipc.ProtocolInfo;
import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.security.KerberosInfo;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenInfo;
/**
* This class defines the actual protocol used to communicate with the
* DN via RPC using writable types.
* The parameters in the methods which are specified in the
* package are separate from those used internally in the DN and DFSClient
* and hence need to be converted using {@link ClientDatanodeProtocolTranslatorR23}
* and {@link ClientDatanodeProtocolServerSideTranslatorR23}.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
@KerberosInfo(
serverPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
@TokenInfo(BlockTokenSelector.class)
@ProtocolInfo(protocolName = HdfsConstants.CLIENT_DATANODE_PROTOCOL_NAME)
public interface ClientDatanodeWireProtocol extends VersionedProtocol {
public static final Log LOG =
LogFactory.getLog(ClientDatanodeWireProtocol.class);
/**
* The rules for changing this protocol are the same as that for
* {@link ClientNamenodeWireProtocol} - see that java file for details.
* 9: Added deleteBlockPool method
* 10 Moved the R23 protocol
*/
public static final long versionID = 10L;
/**
* The specification of this method matches that of
*
* {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol
* #getReplicaVisibleLength(org.apache.hadoop.hdfs.protocol.ExtendedBlock)}
*/
long getReplicaVisibleLength(ExtendedBlockWritable b) throws IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol#refreshNamenodes()}
*/
void refreshNamenodes() throws IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol#deleteBlockPool(String, boolean)}
*/
void deleteBlockPool(String bpid, boolean force) throws IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol#getBlockLocalPathInfo(ExtendedBlock, Token)}
*/
BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
Token<BlockTokenIdentifier> token) throws IOException;
/**
* This method is defined to get the protocol signature using
* the R23 protocol - hence we have added the suffix of 2 to the method name
* to avoid conflict.
*/
public org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable
getProtocolSignature2(String protocol,
long clientVersion,
int clientMethodsHash) throws IOException;
}

View File

@ -1,463 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.FileNotFoundException;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.Options.Rename;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
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.io.EnumSetWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.token.Token;
/**
* This class is used on the server side. Calls come across the wire for the
* protocol family of Release 23 onwards. This class translates the R23 data
* types to the native data types used inside the NN as specified in the generic
* ClientProtocol.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class ClientNamenodeProtocolServerSideTranslatorR23 implements
ClientNamenodeWireProtocol {
final private ClientProtocol server;
/**
* Constructor
*
* @param server - the NN server
* @throws IOException
*/
public ClientNamenodeProtocolServerSideTranslatorR23(ClientProtocol server)
throws IOException {
this.server = server;
}
/**
* The client side will redirect getProtocolSignature to
* getProtocolSignature2.
*
* However the RPC layer below on the Server side will call getProtocolVersion
* and possibly in the future getProtocolSignature. Hence we still implement
* it even though the end client's call will never reach here.
*/
@Override
public ProtocolSignature getProtocolSignature(String protocol,
long clientVersion, int clientMethodsHash) throws IOException {
/**
* Don't forward this to the server. The protocol version and signature is
* that of {@link ClientNamenodeProtocol}
*
*/
if (!protocol.equals(RPC.getProtocolName(
ClientNamenodeWireProtocol.class))) {
throw new IOException("Namenode Serverside implements " +
RPC.getProtocolName(ClientNamenodeWireProtocol.class) +
". The following requested protocol is unknown: " + protocol);
}
return ProtocolSignature.getProtocolSignature(clientMethodsHash,
ClientNamenodeWireProtocol.versionID,
ClientNamenodeWireProtocol.class);
}
@Override
public ProtocolSignatureWritable
getProtocolSignature2(
String protocol, long clientVersion, int clientMethodsHash)
throws IOException {
/**
* Don't forward this to the server. The protocol version and signature is
* that of {@link ClientNamenodeProtocol}
*
*/
return ProtocolSignatureWritable.convert(
this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
}
@Override
public long getProtocolVersion(String protocol, long clientVersion)
throws IOException {
if (protocol.equals(RPC.getProtocolName(
ClientNamenodeWireProtocol.class))) {
return ClientNamenodeWireProtocol.versionID;
}
throw new IOException("Namenode Serverside implements " +
RPC.getProtocolName(ClientNamenodeWireProtocol.class) +
". The following requested protocol is unknown: " + protocol);
}
@Override
public LocatedBlocksWritable getBlockLocations(
String src, long offset, long length)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
return LocatedBlocksWritable.convertLocatedBlocks(
server.getBlockLocations(src, offset, length));
}
@Override
public FsServerDefaultsWritable getServerDefaults() throws IOException {
return FsServerDefaultsWritable.convert(server.getServerDefaults());
}
@Override
public void create(String src, FsPermissionWritable masked, String clientName,
EnumSetWritable<CreateFlag> flag, boolean createParent,
short replication, long blockSize) throws AccessControlException,
AlreadyBeingCreatedException, DSQuotaExceededException,
FileAlreadyExistsException, FileNotFoundException,
NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
UnresolvedLinkException, IOException {
server.create(src, FsPermissionWritable.convertPermission(masked),
clientName, flag, createParent, replication, blockSize);
}
@Override
public LocatedBlockWritable append(String src, String clientName)
throws AccessControlException, DSQuotaExceededException,
FileNotFoundException, SafeModeException, UnresolvedLinkException,
IOException {
return LocatedBlockWritable.convertLocatedBlock(
server.append(src, clientName));
}
@Override
public boolean setReplication(String src, short replication)
throws AccessControlException, DSQuotaExceededException,
FileNotFoundException, SafeModeException, UnresolvedLinkException,
IOException {
return server.setReplication(src, replication);
}
@Override
public void setPermission(String src, FsPermissionWritable permission)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException {
server.setPermission(src,
FsPermissionWritable.convertPermission(permission));
}
@Override
public void setOwner(String src, String username, String groupname)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException {
server.setOwner(src, username, groupname);
}
@Override
public void abandonBlock(ExtendedBlockWritable b, String src, String holder)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
server.abandonBlock(
ExtendedBlockWritable.convertExtendedBlock(b), src, holder);
}
@Override
public LocatedBlockWritable addBlock(String src, String clientName,
ExtendedBlockWritable previous, DatanodeInfoWritable[] excludeNodes)
throws AccessControlException, FileNotFoundException,
NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
IOException {
return LocatedBlockWritable.convertLocatedBlock(
server.addBlock(src, clientName,
ExtendedBlockWritable.convertExtendedBlock(previous),
DatanodeInfoWritable.convertDatanodeInfo(excludeNodes)));
}
@Override
public LocatedBlockWritable getAdditionalDatanode(String src, ExtendedBlockWritable blk,
DatanodeInfoWritable[] existings, DatanodeInfoWritable[] excludes,
int numAdditionalNodes, String clientName) throws AccessControlException,
FileNotFoundException, SafeModeException, UnresolvedLinkException,
IOException {
return LocatedBlockWritable.convertLocatedBlock(
server.getAdditionalDatanode(src,
ExtendedBlockWritable.convertExtendedBlock(blk),
DatanodeInfoWritable.convertDatanodeInfo(existings),
DatanodeInfoWritable.convertDatanodeInfo(excludes),
numAdditionalNodes, clientName));
}
@Override
public boolean complete(String src, String clientName, ExtendedBlockWritable last)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException {
return server.complete(src, clientName,
ExtendedBlockWritable.convertExtendedBlock(last));
}
@Override
public void reportBadBlocks(LocatedBlockWritable[] blocks) throws IOException {
server.reportBadBlocks(LocatedBlockWritable.convertLocatedBlock(blocks));
}
@Override
public boolean rename(String src, String dst) throws UnresolvedLinkException,
IOException {
return server.rename(src, dst);
}
@Override
public void concat(String trg, String[] srcs) throws IOException,
UnresolvedLinkException {
server.concat(trg, srcs);
}
@Override
public void rename2(String src, String dst, Rename... options)
throws AccessControlException, DSQuotaExceededException,
FileAlreadyExistsException, FileNotFoundException,
NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
UnresolvedLinkException, IOException {
server.rename2(src, dst, options);
}
@Override
public boolean delete(String src, boolean recursive)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException {
return server.delete(src, recursive);
}
@Override
public boolean mkdirs(String src, FsPermissionWritable masked, boolean createParent)
throws AccessControlException, FileAlreadyExistsException,
FileNotFoundException, NSQuotaExceededException,
ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
IOException {
return server.mkdirs(src, FsPermissionWritable.convertPermission(masked),
createParent);
}
@Override
public DirectoryListingWritable getListing(String src, byte[] startAfter,
boolean needLocation) throws AccessControlException,
FileNotFoundException, UnresolvedLinkException, IOException {
return DirectoryListingWritable.convertDirectoryListing(
server.getListing(src, startAfter, needLocation));
}
@Override
public void renewLease(String clientName) throws AccessControlException,
IOException {
server.renewLease(clientName);
}
@Override
public boolean recoverLease(String src, String clientName) throws IOException {
return server.recoverLease(src, clientName);
}
@Override
public long[] getStats() throws IOException {
return server.getStats();
}
@Override
public DatanodeInfoWritable[] getDatanodeReport(DatanodeReportType type)
throws IOException {
return DatanodeInfoWritable
.convertDatanodeInfo(server.getDatanodeReport(type));
}
@Override
public long getPreferredBlockSize(String filename) throws IOException,
UnresolvedLinkException {
return server.getPreferredBlockSize(filename);
}
@Override
public boolean setSafeMode(SafeModeAction action) throws IOException {
return server.setSafeMode(action);
}
@Override
public void saveNamespace() throws AccessControlException, IOException {
server.saveNamespace();
}
@Override
public boolean restoreFailedStorage(String arg)
throws AccessControlException, IOException {
return server.restoreFailedStorage(arg);
}
@Override
public void refreshNodes() throws IOException {
server.refreshNodes();
}
@Override
public void finalizeUpgrade() throws IOException {
server.finalizeUpgrade();
}
@Override
public UpgradeStatusReportWritable distributedUpgradeProgress(UpgradeAction action)
throws IOException {
return UpgradeStatusReportWritable.convert(
server.distributedUpgradeProgress(action));
}
@Override
public CorruptFileBlocksWritable listCorruptFileBlocks(String path, String cookie)
throws IOException {
return CorruptFileBlocksWritable.convertCorruptFilesBlocks(
server.listCorruptFileBlocks(path, cookie));
}
@Override
public void metaSave(String filename) throws IOException {
server.metaSave(filename);
}
@Override
public HdfsFileStatusWritable getFileInfo(String src) throws AccessControlException,
FileNotFoundException, UnresolvedLinkException, IOException {
return HdfsFileStatusWritable.convertHdfsFileStatus(
server.getFileInfo(src));
}
@Override
public HdfsFileStatusWritable getFileLinkInfo(String src)
throws AccessControlException, UnresolvedLinkException, IOException {
return HdfsFileStatusWritable.convertHdfsFileStatus(
server.getFileLinkInfo(src));
}
@Override
public ContentSummaryWritable getContentSummary(String path)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
return ContentSummaryWritable.convert(server.getContentSummary(path));
}
@Override
public void setQuota(String path, long namespaceQuota, long diskspaceQuota)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
server.setQuota(path, namespaceQuota, diskspaceQuota);
}
@Override
public void fsync(String src, String client) throws AccessControlException,
FileNotFoundException, UnresolvedLinkException, IOException {
server.fsync(src, client);
}
@Override
public void setTimes(String src, long mtime, long atime)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
server.setTimes(src, mtime, atime);
}
@Override
public void createSymlink(String target, String link, FsPermissionWritable dirPerm,
boolean createParent) throws AccessControlException,
FileAlreadyExistsException, FileNotFoundException,
ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
IOException {
server.createSymlink(target, link, FsPermissionWritable.convertPermission(dirPerm),
createParent);
}
@Override
public String getLinkTarget(String path) throws AccessControlException,
FileNotFoundException, IOException {
return server.getLinkTarget(path);
}
@Override
public LocatedBlockWritable updateBlockForPipeline(ExtendedBlockWritable block,
String clientName) throws IOException {
return LocatedBlockWritable.convertLocatedBlock(
server.updateBlockForPipeline(
ExtendedBlockWritable.convertExtendedBlock(block), clientName));
}
@Override
public void updatePipeline(String clientName, ExtendedBlockWritable oldBlock,
ExtendedBlockWritable newBlock, DatanodeIDWritable[] newNodes)
throws IOException {
server.updatePipeline(clientName,
ExtendedBlockWritable.convertExtendedBlock(oldBlock),
ExtendedBlockWritable.convertExtendedBlock(newBlock),
DatanodeIDWritable.convertDatanodeID(newNodes));
}
@Override
public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
throws IOException {
return server.getDelegationToken(renewer);
}
@Override
public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
throws IOException {
return server.renewDelegationToken(token);
}
@Override
public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
throws IOException {
server.cancelDelegationToken(token);
}
@Override
public void setBalancerBandwidth(long bandwidth) throws IOException {
server.setBalancerBandwidth(bandwidth);
}
}

View File

@ -1,480 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.Closeable;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
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.ParentNotDirectoryException;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.fs.Options.Rename;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
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;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
import org.apache.hadoop.io.EnumSetWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.retry.RetryPolicies;
import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.io.retry.RetryProxy;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
/**
* This class forwards NN's ClientProtocol calls as RPC calls to the NN server
* while translating from the parameter types used in ClientProtocol to those
* used in protocolR23Compatile.*.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class ClientNamenodeProtocolTranslatorR23 implements
ClientProtocol, Closeable {
final private ClientNamenodeWireProtocol rpcProxy;
private static ClientNamenodeWireProtocol createNamenode(
InetSocketAddress nameNodeAddr, Configuration conf,
UserGroupInformation ugi) throws IOException {
return RPC.getProxy(ClientNamenodeWireProtocol.class,
ClientNamenodeWireProtocol.versionID, nameNodeAddr, ugi, conf,
NetUtils.getSocketFactory(conf, ClientNamenodeWireProtocol.class));
}
/** Create a {@link NameNode} proxy */
static ClientNamenodeWireProtocol createNamenodeWithRetry(
ClientNamenodeWireProtocol rpcNamenode) {
RetryPolicy createPolicy = RetryPolicies
.retryUpToMaximumCountWithFixedSleep(5,
HdfsConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
Map<Class<? extends Exception>, RetryPolicy> remoteExceptionToPolicyMap = new HashMap<Class<? extends Exception>, RetryPolicy>();
remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class,
createPolicy);
Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
new HashMap<Class<? extends Exception>, RetryPolicy>();
exceptionToPolicyMap.put(RemoteException.class, RetryPolicies
.retryByRemoteException(RetryPolicies.TRY_ONCE_THEN_FAIL,
remoteExceptionToPolicyMap));
RetryPolicy methodPolicy = RetryPolicies.retryByException(
RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
Map<String, RetryPolicy> methodNameToPolicyMap = new HashMap<String, RetryPolicy>();
methodNameToPolicyMap.put("create", methodPolicy);
return (ClientNamenodeWireProtocol) RetryProxy.create(
ClientNamenodeWireProtocol.class, rpcNamenode, methodNameToPolicyMap);
}
public ClientNamenodeProtocolTranslatorR23(InetSocketAddress nameNodeAddr,
Configuration conf, UserGroupInformation ugi) throws IOException {
rpcProxy = createNamenodeWithRetry(createNamenode(nameNodeAddr, conf, ugi));
}
public void close() {
RPC.stopProxy(rpcProxy);
}
@Override
public ProtocolSignature getProtocolSignature(String protocolName,
long clientVersion, int clientMethodHash)
throws IOException {
return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
protocolName, clientVersion, clientMethodHash));
}
@Override
public long getProtocolVersion(String protocolName, long clientVersion) throws IOException {
return rpcProxy.getProtocolVersion(protocolName, clientVersion);
}
@Override
public LocatedBlocks getBlockLocations(String src, long offset, long length)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
return LocatedBlocksWritable
.convertLocatedBlocks(rpcProxy.getBlockLocations(src, offset, length));
}
@Override
public FsServerDefaults getServerDefaults() throws IOException {
return FsServerDefaultsWritable
.convert(rpcProxy.getServerDefaults());
}
@Override
public void create(String src, FsPermission masked, String clientName,
EnumSetWritable<CreateFlag> flag, boolean createParent,
short replication, long blockSize) throws AccessControlException,
AlreadyBeingCreatedException, DSQuotaExceededException,
FileAlreadyExistsException, FileNotFoundException,
NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
UnresolvedLinkException, IOException {
rpcProxy.create(src, FsPermissionWritable.convertPermission(masked),
clientName, flag, createParent, replication, blockSize);
}
@Override
public LocatedBlock append(String src, String clientName)
throws AccessControlException, DSQuotaExceededException,
FileNotFoundException, SafeModeException, UnresolvedLinkException,
IOException {
return LocatedBlockWritable
.convertLocatedBlock(rpcProxy.append(src, clientName));
}
@Override
public boolean setReplication(String src, short replication)
throws AccessControlException, DSQuotaExceededException,
FileNotFoundException, SafeModeException, UnresolvedLinkException,
IOException {
return rpcProxy.setReplication(src, replication);
}
@Override
public void setPermission(String src, FsPermission permission)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException {
rpcProxy.setPermission(src,
FsPermissionWritable.convertPermission(permission));
}
@Override
public void setOwner(String src, String username, String groupname)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException {
rpcProxy.setOwner(src, username, groupname);
}
@Override
public void abandonBlock(ExtendedBlock b, String src, String holder)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
rpcProxy.abandonBlock(
ExtendedBlockWritable.convertExtendedBlock(b), src, holder);
}
@Override
public LocatedBlock addBlock(String src, String clientName,
ExtendedBlock previous, DatanodeInfo[] excludeNodes)
throws AccessControlException, FileNotFoundException,
NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
IOException {
return LocatedBlockWritable
.convertLocatedBlock(rpcProxy.addBlock(src, clientName,
ExtendedBlockWritable.convertExtendedBlock(previous),
DatanodeInfoWritable.convertDatanodeInfo(excludeNodes)));
}
@Override
public LocatedBlock getAdditionalDatanode(String src, ExtendedBlock blk,
DatanodeInfo[] existings, DatanodeInfo[] excludes,
int numAdditionalNodes, String clientName) throws AccessControlException,
FileNotFoundException, SafeModeException, UnresolvedLinkException,
IOException {
return LocatedBlockWritable
.convertLocatedBlock(rpcProxy.getAdditionalDatanode(src,
ExtendedBlockWritable.convertExtendedBlock(blk),
DatanodeInfoWritable.convertDatanodeInfo(existings),
DatanodeInfoWritable.convertDatanodeInfo(excludes),
numAdditionalNodes, clientName));
}
@Override
public boolean complete(String src, String clientName, ExtendedBlock last)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException {
return rpcProxy.complete(src, clientName,
ExtendedBlockWritable.convertExtendedBlock(last));
}
@Override
public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
rpcProxy.reportBadBlocks(LocatedBlockWritable.convertLocatedBlock(blocks));
}
@Override
public boolean rename(String src, String dst) throws UnresolvedLinkException,
IOException {
return rpcProxy.rename(src, dst);
}
@Override
public void concat(String trg, String[] srcs) throws IOException,
UnresolvedLinkException {
rpcProxy.concat(trg, srcs);
}
@Override
public void rename2(String src, String dst, Rename... options)
throws AccessControlException, DSQuotaExceededException,
FileAlreadyExistsException, FileNotFoundException,
NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
UnresolvedLinkException, IOException {
rpcProxy.rename2(src, dst, options);
}
@Override
public boolean delete(String src, boolean recursive)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException {
return rpcProxy.delete(src, recursive);
}
@Override
public boolean mkdirs(String src, FsPermission masked, boolean createParent)
throws AccessControlException, FileAlreadyExistsException,
FileNotFoundException, NSQuotaExceededException,
ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
IOException {
return rpcProxy.mkdirs(src,
FsPermissionWritable.convertPermission(masked), createParent);
}
@Override
public DirectoryListing getListing(String src, byte[] startAfter,
boolean needLocation) throws AccessControlException,
FileNotFoundException, UnresolvedLinkException, IOException {
return DirectoryListingWritable.convertDirectoryListing(
rpcProxy.getListing(src, startAfter, needLocation));
}
@Override
public void renewLease(String clientName) throws AccessControlException,
IOException {
rpcProxy.renewLease(clientName);
}
@Override
public boolean recoverLease(String src, String clientName) throws IOException {
return rpcProxy.recoverLease(src, clientName);
}
@Override
public long[] getStats() throws IOException {
return rpcProxy.getStats();
}
@Override
public DatanodeInfo[] getDatanodeReport(DatanodeReportType type)
throws IOException {
return DatanodeInfoWritable.convertDatanodeInfo(
rpcProxy.getDatanodeReport(type));
}
@Override
public long getPreferredBlockSize(String filename) throws IOException,
UnresolvedLinkException {
return rpcProxy.getPreferredBlockSize(filename);
}
@Override
public boolean setSafeMode(SafeModeAction action) throws IOException {
return rpcProxy.setSafeMode(action);
}
@Override
public void saveNamespace() throws AccessControlException, IOException {
rpcProxy.saveNamespace();
}
@Override
public boolean restoreFailedStorage(String arg)
throws AccessControlException, IOException{
return rpcProxy.restoreFailedStorage(arg);
}
@Override
public void refreshNodes() throws IOException {
rpcProxy.refreshNodes();
}
@Override
public void finalizeUpgrade() throws IOException {
rpcProxy.finalizeUpgrade();
}
@Override
public UpgradeStatusReport distributedUpgradeProgress(UpgradeAction action)
throws IOException {
return UpgradeStatusReportWritable.convert(
rpcProxy.distributedUpgradeProgress(action));
}
@Override
public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
throws IOException {
return CorruptFileBlocksWritable.convertCorruptFileBlocks(
rpcProxy.listCorruptFileBlocks(path, cookie));
}
@Override
public void metaSave(String filename) throws IOException {
rpcProxy.metaSave(filename);
}
@Override
public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
FileNotFoundException, UnresolvedLinkException, IOException {
return HdfsFileStatusWritable.convertHdfsFileStatus(
rpcProxy.getFileInfo(src));
}
@Override
public HdfsFileStatus getFileLinkInfo(String src)
throws AccessControlException, UnresolvedLinkException, IOException {
return HdfsFileStatusWritable
.convertHdfsFileStatus(rpcProxy.getFileLinkInfo(src));
}
@Override
public ContentSummary getContentSummary(String path)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
return ContentSummaryWritable
.convert(rpcProxy.getContentSummary(path));
}
@Override
public void setQuota(String path, long namespaceQuota, long diskspaceQuota)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
rpcProxy.setQuota(path, namespaceQuota, diskspaceQuota);
}
@Override
public void fsync(String src, String client) throws AccessControlException,
FileNotFoundException, UnresolvedLinkException, IOException {
rpcProxy.fsync(src, client);
}
@Override
public void setTimes(String src, long mtime, long atime)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException {
rpcProxy.setTimes(src, mtime, atime);
}
@Override
public void createSymlink(String target, String link, FsPermission dirPerm,
boolean createParent) throws AccessControlException,
FileAlreadyExistsException, FileNotFoundException,
ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
IOException {
rpcProxy.createSymlink(target, link,
FsPermissionWritable.convertPermission(dirPerm), createParent);
}
@Override
public String getLinkTarget(String path) throws AccessControlException,
FileNotFoundException, IOException {
return rpcProxy.getLinkTarget(path);
}
@Override
public LocatedBlock updateBlockForPipeline(ExtendedBlock block,
String clientName) throws IOException {
return LocatedBlockWritable.convertLocatedBlock(
rpcProxy.updateBlockForPipeline(
ExtendedBlockWritable.convertExtendedBlock(block), clientName));
}
@Override
public void updatePipeline(String clientName, ExtendedBlock oldBlock,
ExtendedBlock newBlock, DatanodeID[] newNodes) throws IOException {
rpcProxy.updatePipeline(clientName,
ExtendedBlockWritable.convertExtendedBlock(oldBlock),
ExtendedBlockWritable.convertExtendedBlock(newBlock),
DatanodeIDWritable.convertDatanodeID(newNodes));
}
@Override
public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
throws IOException {
return rpcProxy.getDelegationToken(renewer);
}
@Override
public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
throws IOException {
return rpcProxy.renewDelegationToken(token);
}
@Override
public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
throws IOException {
rpcProxy.cancelDelegationToken(token);
}
@Override
public void setBalancerBandwidth(long bandwidth) throws IOException {
rpcProxy.setBalancerBandwidth(bandwidth);
}
}

View File

@ -1,478 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.FileNotFoundException;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.Options;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.UpgradeAction;
import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
import org.apache.hadoop.io.EnumSetWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.KerberosInfo;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenInfo;
import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
import org.apache.hadoop.ipc.ProtocolInfo;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
/**********************************************************************
* This class defines the actual protocol used to communicate with the
* NN via RPC using writable types.
* The parameters in the methods which are specified in the
* package are separate from those used internally in the NN and DFSClient
* and hence need to be converted using {@link ClientNamenodeProtocolTranslatorR23}
* and {@link ClientNamenodeProtocolServerSideTranslatorR23}.
*
**********************************************************************/
@InterfaceAudience.Private
@InterfaceStability.Stable
@KerberosInfo(
serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
@TokenInfo(DelegationTokenSelector.class)
@ProtocolInfo(protocolName = HdfsConstants.CLIENT_NAMENODE_PROTOCOL_NAME)
public interface ClientNamenodeWireProtocol extends VersionedProtocol {
/**
* Changes to the protocol:
*
* Do NOT change a method's signature (ie name, parameters, parameter types
* or exceptions thrown). If you need to make changes then ADD new methods and
* new data types.
* Hence if you maintain compatibility you will NOT have to change
* the version number below. The version number is changed ONLY
* if you break compatibility (which is a big deal).
* Hence the version number is really a Major Version Number.
*
* The log of historical changes prior to 69 can be retrieved from the svn.
* ALL changes since version 69L are recorded.
* Version number is changed ONLY for Incompatible changes.
* (note previously we used to change version number for both
* compatible and incompatible changes).
* 69: Eliminate overloaded method names. (Compatible)
* 70: Separation of Datatypes - the client namenode protocol is implemented
* in this class instead of in
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol}
* as was done prior to version 70.
*/
public static final long versionID = 70L;
///////////////////////////////////////
// File contents
///////////////////////////////////////
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getBlockLocations}
*/
public LocatedBlocksWritable getBlockLocations(String src,
long offset,
long length)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getServerDefaults()}
*/
public FsServerDefaultsWritable getServerDefaults() throws IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#create(String,
* org.apache.hadoop.fs.permission.FsPermission, String,
* EnumSetWritable, boolean, short, long)}
*/
public void create(String src, FsPermissionWritable masked, String clientName,
EnumSetWritable<CreateFlag> flag, boolean createParent,
short replication, long blockSize) throws AccessControlException,
AlreadyBeingCreatedException, DSQuotaExceededException,
FileAlreadyExistsException, FileNotFoundException,
NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
UnresolvedLinkException, IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#append(String, String)}
*/
public LocatedBlockWritable append(String src, String clientName)
throws AccessControlException, DSQuotaExceededException,
FileNotFoundException, SafeModeException, UnresolvedLinkException,
IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setReplication(String, short)}
*/
public boolean setReplication(String src, short replication)
throws AccessControlException, DSQuotaExceededException,
FileNotFoundException, SafeModeException, UnresolvedLinkException,
IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setPermission(String,
* org.apache.hadoop.fs.permission.FsPermission)}
*/
public void setPermission(String src, FsPermissionWritable permission)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setOwner(String, String, String)}
*/
public void setOwner(String src, String username, String groupname)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#abandonBlock(
* org.apache.hadoop.hdfs.protocol.ExtendedBlock, String, String)}
*/
public void abandonBlock(ExtendedBlockWritable b, String src, String holder)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#addBlock(String,
* String, org.apache.hadoop.hdfs.protocol.ExtendedBlock,
* org.apache.hadoop.hdfs.protocol.DatanodeInfo[])}
*/
public LocatedBlockWritable addBlock(String src, String clientName,
ExtendedBlockWritable previous, DatanodeInfoWritable[] excludeNodes)
throws AccessControlException, FileNotFoundException,
NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getAdditionalDatanode}
*/
public LocatedBlockWritable getAdditionalDatanode(
final String src, final ExtendedBlockWritable blk,
final DatanodeInfoWritable[] existings,
final DatanodeInfoWritable[] excludes,
final int numAdditionalNodes, final String clientName
) throws AccessControlException, FileNotFoundException,
SafeModeException, UnresolvedLinkException, IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#complete}
*/
public boolean complete(
String src, String clientName, ExtendedBlockWritable last)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#reportBadBlocks}
*/
public void reportBadBlocks(LocatedBlockWritable[] blocks) throws IOException;
///////////////////////////////////////
// Namespace management
///////////////////////////////////////
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#rename(String, String)}
*/
public boolean rename(String src, String dst)
throws UnresolvedLinkException, IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#concat(String, String[])}
*/
public void concat(String trg, String[] srcs)
throws IOException, UnresolvedLinkException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#rename2}
*/
public void rename2(String src, String dst, Options.Rename... options)
throws AccessControlException, DSQuotaExceededException,
FileAlreadyExistsException, FileNotFoundException,
NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
UnresolvedLinkException, IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#delete(String, boolean)}
*/
public boolean delete(String src, boolean recursive)
throws AccessControlException, FileNotFoundException, SafeModeException,
UnresolvedLinkException, IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#mkdirs}
*/
public boolean mkdirs(
String src, FsPermissionWritable masked, boolean createParent)
throws AccessControlException, FileAlreadyExistsException,
FileNotFoundException, NSQuotaExceededException,
ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getListing}
*/
public DirectoryListingWritable getListing(String src,
byte[] startAfter,
boolean needLocation)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException;
///////////////////////////////////////
// System issues and management
///////////////////////////////////////
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#renewLease(String)}
*/
public void renewLease(String clientName) throws AccessControlException,
IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#recoverLease(String, String)}
*/
public boolean recoverLease(String src, String clientName) throws IOException;
public int GET_STATS_CAPACITY_IDX = 0;
public int GET_STATS_USED_IDX = 1;
public int GET_STATS_REMAINING_IDX = 2;
public int GET_STATS_UNDER_REPLICATED_IDX = 3;
public int GET_STATS_CORRUPT_BLOCKS_IDX = 4;
public int GET_STATS_MISSING_BLOCKS_IDX = 5;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getStats()}
*/
public long[] getStats() throws IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getDatanodeReport}
*/
public DatanodeInfoWritable[] getDatanodeReport(
HdfsConstants.DatanodeReportType type)
throws IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getPreferredBlockSize}
*/
public long getPreferredBlockSize(String filename)
throws IOException, UnresolvedLinkException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction)}
*/
public boolean setSafeMode(HdfsConstants.SafeModeAction action)
throws IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#saveNamespace()}
*/
public void saveNamespace() throws AccessControlException, IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#restoreFailedStorage(String)}
*/
public boolean restoreFailedStorage(String arg)
throws AccessControlException, IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#refreshNodes()}
*/
public void refreshNodes() throws IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#finalizeUpgrade()}
*/
public void finalizeUpgrade() throws IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#distributedUpgradeProgress}
*/
public UpgradeStatusReportWritable distributedUpgradeProgress(
UpgradeAction action)
throws IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#listCorruptFileBlocks(String, String)}
*/
public CorruptFileBlocksWritable
listCorruptFileBlocks(String path, String cookie)
throws IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#metaSave(String)}
*/
public void metaSave(String filename) throws IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setBalancerBandwidth(long)}
*/
public void setBalancerBandwidth(long bandwidth) throws IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getFileInfo(String)}
*/
public HdfsFileStatusWritable getFileInfo(String src)
throws AccessControlException,
FileNotFoundException, UnresolvedLinkException, IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getFileLinkInfo(String)}
*/
public HdfsFileStatusWritable getFileLinkInfo(String src)
throws AccessControlException, UnresolvedLinkException, IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getContentSummary(String)}
*/
public ContentSummaryWritable getContentSummary(String path)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setQuota(String, long, long)}
*/
public void setQuota(String path, long namespaceQuota, long diskspaceQuota)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#fsync(String, String)}
*/
public void fsync(String src, String client)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#setTimes(String, long, long)}
*/
public void setTimes(String src, long mtime, long atime)
throws AccessControlException, FileNotFoundException,
UnresolvedLinkException, IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#createSymlink}
*/
public void createSymlink(
String target, String link, FsPermissionWritable dirPerm,
boolean createParent) throws AccessControlException,
FileAlreadyExistsException, FileNotFoundException,
ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getLinkTarget(String)}
*/
public String getLinkTarget(String path) throws AccessControlException,
FileNotFoundException, IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#updateBlockForPipeline}
*/
public LocatedBlockWritable updateBlockForPipeline(
ExtendedBlockWritable block, String clientName) throws IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#updatePipeline}
*/
public void updatePipeline(String clientName, ExtendedBlockWritable oldBlock,
ExtendedBlockWritable newBlock, DatanodeIDWritable[] newNodes)
throws IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#getDelegationToken(Text)}
*/
public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
throws IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#renewDelegationToken(Token)}
*/
public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
throws IOException;
/**
* The specification of this method matches that of
* {@link org.apache.hadoop.hdfs.protocol.ClientProtocol#cancelDelegationToken(Token)}
*/
public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
throws IOException;
/**
* This method is defined to get the protocol signature using
* the R23 protocol - hence we have added the suffix of 2 the method name
* to avoid conflict.
*/
public org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable
getProtocolSignature2(String protocol,
long clientVersion,
int clientMethodsHash) throws IOException;
}

View File

@ -1,184 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.Writable;
/** Store the summary of a content (a directory or a file). */
@InterfaceAudience.Public
@InterfaceStability.Stable
public class ContentSummaryWritable implements Writable{
private long length;
private long fileCount;
private long directoryCount;
private long quota;
private long spaceConsumed;
private long spaceQuota;
public static org.apache.hadoop.fs.ContentSummary convert(ContentSummaryWritable cs) {
if (cs == null) return null;
return new org.apache.hadoop.fs.ContentSummary(
cs.getLength(), cs.getFileCount(), cs.getDirectoryCount(), cs.getQuota(),
cs.getSpaceConsumed(), cs.getSpaceQuota());
}
public static ContentSummaryWritable convert(org.apache.hadoop.fs.ContentSummary cs) {
if (cs == null) return null;
return new ContentSummaryWritable(
cs.getLength(), cs.getFileCount(), cs.getDirectoryCount(), cs.getQuota(),
cs.getSpaceConsumed(), cs.getSpaceQuota());
}
/** Constructor */
public ContentSummaryWritable() {}
/** Constructor */
public ContentSummaryWritable(long length, long fileCount, long directoryCount) {
this(length, fileCount, directoryCount, -1L, length, -1L);
}
/** Constructor */
public ContentSummaryWritable(
long length, long fileCount, long directoryCount, long quota,
long spaceConsumed, long spaceQuota) {
this.length = length;
this.fileCount = fileCount;
this.directoryCount = directoryCount;
this.quota = quota;
this.spaceConsumed = spaceConsumed;
this.spaceQuota = spaceQuota;
}
/** @return the length */
public long getLength() {return length;}
/** @return the directory count */
public long getDirectoryCount() {return directoryCount;}
/** @return the file count */
public long getFileCount() {return fileCount;}
/** Return the directory quota */
public long getQuota() {return quota;}
/** Retuns (disk) space consumed */
public long getSpaceConsumed() {return spaceConsumed;}
/** Returns (disk) space quota */
public long getSpaceQuota() {return spaceQuota;}
@InterfaceAudience.Private
@Override
public void write(DataOutput out) throws IOException {
out.writeLong(length);
out.writeLong(fileCount);
out.writeLong(directoryCount);
out.writeLong(quota);
out.writeLong(spaceConsumed);
out.writeLong(spaceQuota);
}
@InterfaceAudience.Private
@Override
public void readFields(DataInput in) throws IOException {
this.length = in.readLong();
this.fileCount = in.readLong();
this.directoryCount = in.readLong();
this.quota = in.readLong();
this.spaceConsumed = in.readLong();
this.spaceQuota = in.readLong();
}
/**
* Output format:
* <----12----> <----12----> <-------18------->
* DIR_COUNT FILE_COUNT CONTENT_SIZE FILE_NAME
*/
private static final String STRING_FORMAT = "%12d %12d %18d ";
/**
* Output format:
* <----12----> <----15----> <----15----> <----15----> <----12----> <----12----> <-------18------->
* QUOTA REMAINING_QUATA SPACE_QUOTA SPACE_QUOTA_REM DIR_COUNT FILE_COUNT CONTENT_SIZE FILE_NAME
*/
private static final String QUOTA_STRING_FORMAT = "%12s %15s ";
private static final String SPACE_QUOTA_STRING_FORMAT = "%15s %15s ";
/** The header string */
private static final String HEADER = String.format(
STRING_FORMAT.replace('d', 's'), "directories", "files", "bytes");
private static final String QUOTA_HEADER = String.format(
QUOTA_STRING_FORMAT + SPACE_QUOTA_STRING_FORMAT,
"quota", "remaining quota", "space quota", "reamaining quota") +
HEADER;
/** Return the header of the output.
* if qOption is false, output directory count, file count, and content size;
* if qOption is true, output quota and remaining quota as well.
*
* @param qOption a flag indicating if quota needs to be printed or not
* @return the header of the output
*/
public static String getHeader(boolean qOption) {
return qOption ? QUOTA_HEADER : HEADER;
}
@Override
public String toString() {
return toString(true);
}
/** Return the string representation of the object in the output format.
* if qOption is false, output directory count, file count, and content size;
* if qOption is true, output quota and remaining quota as well.
*
* @param qOption a flag indicating if quota needs to be printed or not
* @return the string representation of the object
*/
public String toString(boolean qOption) {
String prefix = "";
if (qOption) {
String quotaStr = "none";
String quotaRem = "inf";
String spaceQuotaStr = "none";
String spaceQuotaRem = "inf";
if (quota>0) {
quotaStr = Long.toString(quota);
quotaRem = Long.toString(quota-(directoryCount+fileCount));
}
if (spaceQuota>0) {
spaceQuotaStr = Long.toString(spaceQuota);
spaceQuotaRem = Long.toString(spaceQuota - spaceConsumed);
}
prefix = String.format(QUOTA_STRING_FORMAT + SPACE_QUOTA_STRING_FORMAT,
quotaStr, quotaRem, spaceQuotaStr, spaceQuotaRem);
}
return prefix + String.format(STRING_FORMAT, directoryCount,
fileCount, length);
}
}

View File

@ -1,88 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.Text;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
/**
* Contains a list of paths corresponding to corrupt files and a cookie
* used for iterative calls to NameNode.listCorruptFileBlocks.
*
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class CorruptFileBlocksWritable implements Writable {
private String[] files;
private String cookie;
static public org.apache.hadoop.hdfs.protocol.CorruptFileBlocks
convertCorruptFileBlocks(CorruptFileBlocksWritable c) {
if (c == null) return null;
return new org.apache.hadoop.hdfs.protocol.CorruptFileBlocks(
c.getFiles(), c.getCookie());
}
public static CorruptFileBlocksWritable convertCorruptFilesBlocks(
org.apache.hadoop.hdfs.protocol.CorruptFileBlocks c) {
if (c == null) return null;
return new CorruptFileBlocksWritable(c.getFiles(), c.getCookie());
}
public CorruptFileBlocksWritable() {
this(new String[0], "");
}
public CorruptFileBlocksWritable(String[] files, String cookie) {
this.files = files;
this.cookie = cookie;
}
public String[] getFiles() {
return files;
}
public String getCookie() {
return cookie;
}
@Override
public void readFields(DataInput in) throws IOException {
int fileCount = in.readInt();
files = new String[fileCount];
for (int i = 0; i < fileCount; i++) {
files[i] = Text.readString(in);
}
cookie = Text.readString(in);
}
@Override
public void write(DataOutput out) throws IOException {
out.writeInt(files.length);
for (int i = 0; i < files.length; i++) {
Text.writeString(out, files[i]);
}
Text.writeString(out, cookie);
}
}

View File

@ -1,209 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.DeprecatedUTF8;
import org.apache.hadoop.io.Writable;
/**
* DatanodeID is composed of the data node
* name (hostname:portNumber) and the data storage ID,
* which it currently represents.
*
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class DatanodeIDWritable implements Writable {
public static final DatanodeIDWritable[] EMPTY_ARRAY = {};
public String name; /// hostname:portNumber
public String storageID; /// unique per cluster storageID
protected int infoPort; /// the port where the infoserver is running
public int ipcPort; /// the port where the ipc server is running
static public DatanodeIDWritable[]
convertDatanodeID(org.apache.hadoop.hdfs.protocol.DatanodeID[] did) {
if (did == null) return null;
final int len = did.length;
DatanodeIDWritable[] result = new DatanodeIDWritable[len];
for (int i = 0; i < len; ++i) {
result[i] = convertDatanodeID(did[i]);
}
return result;
}
static public org.apache.hadoop.hdfs.protocol.DatanodeID[]
convertDatanodeID(DatanodeIDWritable[] did) {
if (did == null) return null;
final int len = did.length;
org.apache.hadoop.hdfs.protocol.DatanodeID[] result = new org.apache.hadoop.hdfs.protocol.DatanodeID[len];
for (int i = 0; i < len; ++i) {
result[i] = convertDatanodeID(did[i]);
}
return result;
}
static public org.apache.hadoop.hdfs.protocol.DatanodeID convertDatanodeID(
DatanodeIDWritable did) {
if (did == null) return null;
return new org.apache.hadoop.hdfs.protocol.DatanodeID(
did.getName(), did.getStorageID(), did.getInfoPort(), did.getIpcPort());
}
public static DatanodeIDWritable convertDatanodeID(org.apache.hadoop.hdfs.protocol.DatanodeID from) {
return new DatanodeIDWritable(from.getName(),
from.getStorageID(),
from.getInfoPort(),
from.getIpcPort());
}
/** Equivalent to DatanodeID(""). */
public DatanodeIDWritable() {this("");}
/** Equivalent to DatanodeID(nodeName, "", -1, -1). */
public DatanodeIDWritable(String nodeName) {this(nodeName, "", -1, -1);}
/**
* DatanodeID copy constructor
*
* @param from
*/
public DatanodeIDWritable(DatanodeIDWritable from) {
this(from.getName(),
from.getStorageID(),
from.getInfoPort(),
from.getIpcPort());
}
/**
* Create DatanodeID
* @param nodeName (hostname:portNumber)
* @param storageID data storage ID
* @param infoPort info server port
* @param ipcPort ipc server port
*/
public DatanodeIDWritable(String nodeName, String storageID,
int infoPort, int ipcPort) {
this.name = nodeName;
this.storageID = storageID;
this.infoPort = infoPort;
this.ipcPort = ipcPort;
}
public void setName(String name) {
this.name = name;
}
public void setInfoPort(int infoPort) {
this.infoPort = infoPort;
}
public void setIpcPort(int ipcPort) {
this.ipcPort = ipcPort;
}
/**
* @return hostname:portNumber.
*/
public String getName() {
return name;
}
/**
* @return data storage ID.
*/
public String getStorageID() {
return this.storageID;
}
/**
* @return infoPort (the port at which the HTTP server bound to)
*/
public int getInfoPort() {
return infoPort;
}
/**
* @return ipcPort (the port at which the IPC server bound to)
*/
public int getIpcPort() {
return ipcPort;
}
/**
* sets the data storage ID.
*/
public void setStorageID(String storageID) {
this.storageID = storageID;
}
/**
* @return hostname and no :portNumber.
*/
public String getHost() {
int colon = name.indexOf(":");
if (colon < 0) {
return name;
} else {
return name.substring(0, colon);
}
}
public int getPort() {
int colon = name.indexOf(":");
if (colon < 0) {
return 50010; // default port.
}
return Integer.parseInt(name.substring(colon+1));
}
public String toString() {
return name;
}
/////////////////////////////////////////////////
// Writable
/////////////////////////////////////////////////
@Override
public void write(DataOutput out) throws IOException {
DeprecatedUTF8.writeString(out, name);
DeprecatedUTF8.writeString(out, storageID);
out.writeShort(infoPort);
}
@Override
public void readFields(DataInput in) throws IOException {
name = DeprecatedUTF8.readString(in);
storageID = DeprecatedUTF8.readString(in);
// the infoPort read could be negative, if the port is a large number (more
// than 15 bits in storage size (but less than 16 bits).
// So chop off the first two bytes (and hence the signed bits) before
// setting the field.
this.infoPort = in.readShort() & 0x0000ffff;
}
}

View File

@ -1,334 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.net.NodeBase;
import org.apache.hadoop.HadoopIllegalArgumentException;
/**
* DatanodeInfo represents the status of a DataNode.
* This object is used for communication in the
* Datanode Protocol and the Client Protocol.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class DatanodeInfoWritable extends DatanodeIDWritable {
protected long capacity;
protected long dfsUsed;
protected long remaining;
protected long blockPoolUsed;
protected long lastUpdate;
protected int xceiverCount;
protected String location = NetworkTopology.DEFAULT_RACK;
/** HostName as supplied by the datanode during registration as its
* name. Namenode uses datanode IP address as the name.
*/
protected String hostName = null;
// administrative states of a datanode
public enum AdminStates {
NORMAL(DatanodeInfo.AdminStates.NORMAL.toString()),
DECOMMISSION_INPROGRESS(DatanodeInfo.AdminStates.DECOMMISSION_INPROGRESS.toString()),
DECOMMISSIONED(DatanodeInfo.AdminStates.DECOMMISSIONED.toString());
final String value;
AdminStates(final String v) {
this.value = v;
}
public String toString() {
return value;
}
public static AdminStates fromValue(final String value) {
for (AdminStates as : AdminStates.values()) {
if (as.value.equals(value)) return as;
}
throw new HadoopIllegalArgumentException("Unknown Admin State" + value);
}
}
protected AdminStates adminState;
static public DatanodeInfo convertDatanodeInfo(DatanodeInfoWritable di) {
if (di == null) return null;
return new DatanodeInfo(
new org.apache.hadoop.hdfs.protocol.DatanodeID(di.getName(), di.getStorageID(), di.getInfoPort(), di.getIpcPort()),
di.getNetworkLocation(), di.getHostName(),
di.getCapacity(), di.getDfsUsed(), di.getRemaining(),
di.getBlockPoolUsed() , di.getLastUpdate() , di.getXceiverCount() ,
DatanodeInfo.AdminStates.fromValue(di.getAdminState().value));
}
static public DatanodeInfo[] convertDatanodeInfo(DatanodeInfoWritable di[]) {
if (di == null) return null;
DatanodeInfo[] result = new DatanodeInfo[di.length];
for (int i = 0; i < di.length; i++) {
result[i] = convertDatanodeInfo(di[i]);
}
return result;
}
static public DatanodeInfoWritable[] convertDatanodeInfo(DatanodeInfo[] di) {
if (di == null) return null;
DatanodeInfoWritable[] result = new DatanodeInfoWritable[di.length];
for (int i = 0; i < di.length; i++) {
result[i] = new DatanodeInfoWritable(new DatanodeIDWritable(di[i].getName(), di[i].getStorageID(), di[i].getInfoPort(), di[i].getIpcPort()),
di[i].getNetworkLocation(), di[i].getHostName(),
di[i].getCapacity(), di[i].getDfsUsed(), di[i].getRemaining(),
di[i].getBlockPoolUsed() , di[i].getLastUpdate() , di[i].getXceiverCount() ,
AdminStates.fromValue(di[i].getAdminState().toString()));
}
return result;
}
static public DatanodeInfoWritable convertDatanodeInfo(DatanodeInfo di) {
if (di == null) return null;
return new DatanodeInfoWritable(new DatanodeIDWritable(di.getName(),
di.getStorageID(), di.getInfoPort(), di.getIpcPort()),
di.getNetworkLocation(), di.getHostName(), di.getCapacity(),
di.getDfsUsed(), di.getRemaining(), di.getBlockPoolUsed(),
di.getLastUpdate(), di.getXceiverCount(),
AdminStates.fromValue(di.getAdminState().toString()));
}
public DatanodeInfoWritable() {
super();
adminState = null;
}
public DatanodeInfoWritable(DatanodeInfoWritable from) {
super(from);
this.capacity = from.getCapacity();
this.dfsUsed = from.getDfsUsed();
this.remaining = from.getRemaining();
this.blockPoolUsed = from.getBlockPoolUsed();
this.lastUpdate = from.getLastUpdate();
this.xceiverCount = from.getXceiverCount();
this.location = from.getNetworkLocation();
this.adminState = from.adminState;
this.hostName = from.hostName;
}
public DatanodeInfoWritable(DatanodeIDWritable nodeID) {
super(nodeID);
this.capacity = 0L;
this.dfsUsed = 0L;
this.remaining = 0L;
this.blockPoolUsed = 0L;
this.lastUpdate = 0L;
this.xceiverCount = 0;
this.adminState = null;
}
protected DatanodeInfoWritable(DatanodeIDWritable nodeID, String location, String hostName) {
this(nodeID);
this.location = location;
this.hostName = hostName;
}
public DatanodeInfoWritable(DatanodeIDWritable nodeID, String location, String hostName,
final long capacity, final long dfsUsed, final long remaining,
final long blockPoolUsed, final long lastUpdate, final int xceiverCount,
final AdminStates adminState) {
this(nodeID, location, hostName);
this.capacity = capacity;
this.dfsUsed = dfsUsed;
this.remaining = remaining;
this.blockPoolUsed = blockPoolUsed;
this.lastUpdate = lastUpdate;
this.xceiverCount = xceiverCount;
this.adminState = adminState;
}
/** The raw capacity. */
public long getCapacity() { return capacity; }
/** The used space by the data node. */
public long getDfsUsed() { return dfsUsed; }
/** The used space by the block pool on data node. */
public long getBlockPoolUsed() { return blockPoolUsed; }
/** The used space by the data node. */
public long getNonDfsUsed() {
long nonDFSUsed = capacity - dfsUsed - remaining;
return nonDFSUsed < 0 ? 0 : nonDFSUsed;
}
/** The used space by the data node as percentage of present capacity */
public float getDfsUsedPercent() {
return DFSUtil.getPercentUsed(dfsUsed, capacity);
}
/** The raw free space. */
public long getRemaining() { return remaining; }
/** Used space by the block pool as percentage of present capacity */
public float getBlockPoolUsedPercent() {
return DFSUtil.getPercentUsed(blockPoolUsed, capacity);
}
/** The remaining space as percentage of configured capacity. */
public float getRemainingPercent() {
return DFSUtil.getPercentRemaining(remaining, capacity);
}
/** The time when this information was accurate. */
public long getLastUpdate() { return lastUpdate; }
/** number of active connections */
public int getXceiverCount() { return xceiverCount; }
/** Sets raw capacity. */
public void setCapacity(long capacity) {
this.capacity = capacity;
}
/** Sets the used space for the datanode. */
public void setDfsUsed(long dfsUsed) {
this.dfsUsed = dfsUsed;
}
/** Sets raw free space. */
public void setRemaining(long remaining) {
this.remaining = remaining;
}
/** Sets block pool used space */
public void setBlockPoolUsed(long bpUsed) {
this.blockPoolUsed = bpUsed;
}
/** Sets time when this information was accurate. */
public void setLastUpdate(long lastUpdate) {
this.lastUpdate = lastUpdate;
}
/** Sets number of active connections */
public void setXceiverCount(int xceiverCount) {
this.xceiverCount = xceiverCount;
}
/** rack name */
public String getNetworkLocation() {return location;}
/** Sets the rack name */
public void setNetworkLocation(String location) {
this.location = NodeBase.normalize(location);
}
public String getHostName() {
return (hostName == null || hostName.length()==0) ? getHost() : hostName;
}
public void setHostName(String host) {
hostName = host;
}
/**
* Retrieves the admin state of this node.
*/
public AdminStates getAdminState() {
if (adminState == null) {
return AdminStates.NORMAL;
}
return adminState;
}
/**
* Sets the admin state of this node.
*/
protected void setAdminState(AdminStates newState) {
if (newState == AdminStates.NORMAL) {
adminState = null;
}
else {
adminState = newState;
}
}
/////////////////////////////////////////////////
// Writable
/////////////////////////////////////////////////
static { // register a ctor
WritableFactories.setFactory
(DatanodeInfoWritable.class,
new WritableFactory() {
public Writable newInstance() { return new DatanodeInfoWritable(); }
});
}
@Override
public void write(DataOutput out) throws IOException {
super.write(out);
out.writeShort(ipcPort);
out.writeLong(capacity);
out.writeLong(dfsUsed);
out.writeLong(remaining);
out.writeLong(blockPoolUsed);
out.writeLong(lastUpdate);
out.writeInt(xceiverCount);
Text.writeString(out, location);
Text.writeString(out, hostName == null? "" : hostName);
WritableUtils.writeEnum(out, getAdminState());
}
@Override
public void readFields(DataInput in) throws IOException {
super.readFields(in);
this.ipcPort = in.readShort() & 0x0000ffff;
this.capacity = in.readLong();
this.dfsUsed = in.readLong();
this.remaining = in.readLong();
this.blockPoolUsed = in.readLong();
this.lastUpdate = in.readLong();
this.xceiverCount = in.readInt();
this.location = Text.readString(in);
this.hostName = Text.readString(in);
setAdminState(WritableUtils.readEnum(in, AdminStates.class));
}
/** Read a DatanodeInfo */
public static DatanodeInfoWritable read(DataInput in) throws IOException {
final DatanodeInfoWritable d = new DatanodeInfoWritable();
d.readFields(in);
return d;
}
}

View File

@ -1,157 +0,0 @@
/* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
/**
* This class defines a partial listing of a directory to support
* iterative directory listing.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class DirectoryListingWritable implements Writable {
static { // register a ctor
WritableFactories.setFactory
(DirectoryListingWritable.class,
new WritableFactory() {
public Writable newInstance() { return new DirectoryListingWritable(); }
});
}
private HdfsFileStatusWritable[] partialListing;
private int remainingEntries;
public static org.apache.hadoop.hdfs.protocol.DirectoryListing
convertDirectoryListing(DirectoryListingWritable dl) {
if (dl == null) return null;
return new org.apache.hadoop.hdfs.protocol.DirectoryListing(
HdfsFileStatusWritable.convertHdfsFileStatus(
dl.getPartialListing()), dl.getRemainingEntries());
}
public static DirectoryListingWritable convertDirectoryListing(
org.apache.hadoop.hdfs.protocol.DirectoryListing d) {
if (d == null) return null;
return new DirectoryListingWritable(
org.apache.hadoop.hdfs.protocolR23Compatible.HdfsFileStatusWritable.
convertHdfsFileStatus(d.getPartialListing()), d.getRemainingEntries());
}
/**
* default constructor
*/
public DirectoryListingWritable() {
}
/**
* constructor
* @param partialListing a partial listing of a directory
* @param remainingEntries number of entries that are left to be listed
*/
public DirectoryListingWritable(HdfsFileStatusWritable[] partialListing,
int remainingEntries) {
if (partialListing == null) {
throw new IllegalArgumentException("partial listing should not be null");
}
if (partialListing.length == 0 && remainingEntries != 0) {
throw new IllegalArgumentException("Partial listing is empty but " +
"the number of remaining entries is not zero");
}
this.partialListing = partialListing;
this.remainingEntries = remainingEntries;
}
/**
* Get the partial listing of file status
* @return the partial listing of file status
*/
public HdfsFileStatusWritable[] getPartialListing() {
return partialListing;
}
/**
* Get the number of remaining entries that are left to be listed
* @return the number of remaining entries that are left to be listed
*/
public int getRemainingEntries() {
return remainingEntries;
}
/**
* Check if there are more entries that are left to be listed
* @return true if there are more entries that are left to be listed;
* return false otherwise.
*/
public boolean hasMore() {
return remainingEntries != 0;
}
/**
* Get the last name in this list
* @return the last name in the list if it is not empty; otherwise return null
*/
public byte[] getLastName() {
if (partialListing.length == 0) {
return null;
}
return partialListing[partialListing.length-1].getLocalNameInBytes();
}
// Writable interface
@Override
public void readFields(DataInput in) throws IOException {
int numEntries = in.readInt();
partialListing = new HdfsFileStatusWritable[numEntries];
if (numEntries !=0 ) {
boolean hasLocation = in.readBoolean();
for (int i=0; i<numEntries; i++) {
if (hasLocation) {
partialListing[i] = new HdfsLocatedFileStatusWritable();
} else {
partialListing[i] = new HdfsFileStatusWritable();
}
partialListing[i].readFields(in);
}
}
remainingEntries = in.readInt();
}
@Override
public void write(DataOutput out) throws IOException {
out.writeInt(partialListing.length);
if (partialListing.length != 0) {
if (partialListing[0] instanceof HdfsLocatedFileStatusWritable) {
out.writeBoolean(true);
} else {
out.writeBoolean(false);
}
for (HdfsFileStatusWritable fileStatus : partialListing) {
fileStatus.write(out);
}
}
out.writeInt(remainingEntries);
}
}

View File

@ -1,104 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.security.token.block.BlockKey;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
/**
* Object for passing block keys
*/
@InterfaceAudience.Private
public class ExportedBlockKeysWritable implements Writable {
private boolean isBlockTokenEnabled;
private long keyUpdateInterval;
private long tokenLifetime;
private BlockKey currentKey;
private BlockKey[] allKeys;
public ExportedBlockKeysWritable() {
this(false, 0, 0, new BlockKey(), new BlockKey[0]);
}
ExportedBlockKeysWritable(boolean isBlockTokenEnabled, long keyUpdateInterval,
long tokenLifetime, BlockKey currentKey, BlockKey[] allKeys) {
this.isBlockTokenEnabled = isBlockTokenEnabled;
this.keyUpdateInterval = keyUpdateInterval;
this.tokenLifetime = tokenLifetime;
this.currentKey = currentKey == null ? new BlockKey() : currentKey;
this.allKeys = allKeys == null ? new BlockKey[0] : allKeys;
}
// ///////////////////////////////////////////////
// Writable
// ///////////////////////////////////////////////
static { // register a ctor
WritableFactories.setFactory(ExportedBlockKeysWritable.class,
new WritableFactory() {
public Writable newInstance() {
return new ExportedBlockKeysWritable();
}
});
}
@Override
public void write(DataOutput out) throws IOException {
out.writeBoolean(isBlockTokenEnabled);
out.writeLong(keyUpdateInterval);
out.writeLong(tokenLifetime);
currentKey.write(out);
out.writeInt(allKeys.length);
for (int i = 0; i < allKeys.length; i++) {
allKeys[i].write(out);
}
}
@Override
public void readFields(DataInput in) throws IOException {
isBlockTokenEnabled = in.readBoolean();
keyUpdateInterval = in.readLong();
tokenLifetime = in.readLong();
currentKey.readFields(in);
this.allKeys = new BlockKey[in.readInt()];
for (int i = 0; i < allKeys.length; i++) {
allKeys[i] = new BlockKey();
allKeys[i].readFields(in);
}
}
public static ExportedBlockKeysWritable convert(ExportedBlockKeys blockKeys) {
if (blockKeys == null) return null;
return new ExportedBlockKeysWritable(blockKeys.isBlockTokenEnabled(),
blockKeys.getKeyUpdateInterval(), blockKeys.getTokenLifetime(),
blockKeys.getCurrentKey(), blockKeys.getAllKeys());
}
public ExportedBlockKeys convert() {
return new ExportedBlockKeys(isBlockTokenEnabled, keyUpdateInterval,
tokenLifetime, currentKey, allKeys);
}
}

View File

@ -1,121 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.DeprecatedUTF8;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
/**
* Identifies a Block uniquely across the block pools
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class ExtendedBlockWritable implements Writable {
private String poolId;
private long blockId;
private long numBytes;
private long generationStamp;
static { // register a ctor
WritableFactories.setFactory(ExtendedBlockWritable.class, new WritableFactory() {
public Writable newInstance() {
return new ExtendedBlockWritable();
}
});
}
static public org.apache.hadoop.hdfs.protocol.ExtendedBlock convertExtendedBlock(ExtendedBlockWritable eb) {
if (eb == null) return null;
return new org.apache.hadoop.hdfs.protocol.ExtendedBlock( eb.getBlockPoolId(), eb.getBlockId(), eb.getNumBytes(),
eb.getGenerationStamp());
}
public static ExtendedBlockWritable convertExtendedBlock(final org.apache.hadoop.hdfs.protocol.ExtendedBlock b) {
if (b == null) return null;
return new ExtendedBlockWritable(b.getBlockPoolId(),
b.getBlockId(), b.getNumBytes(), b.getGenerationStamp());
}
public ExtendedBlockWritable() {
this(null, 0, 0, 0);
}
public ExtendedBlockWritable(final ExtendedBlockWritable b) {
this(b.poolId, b.blockId, b.numBytes, b.generationStamp);
}
public ExtendedBlockWritable(final String poolId, final long blockId) {
this(poolId, blockId, 0, 0);
}
public ExtendedBlockWritable(final String poolId, final long blkid, final long len,
final long genstamp) {
this.poolId = poolId;
this.blockId = blkid;
this.numBytes = len;
this.generationStamp = genstamp;
}
@Override
public void write(DataOutput out) throws IOException {
DeprecatedUTF8.writeString(out, poolId);
out.writeLong(blockId);
out.writeLong(numBytes);
out.writeLong(generationStamp);
}
@Override
public void readFields(DataInput in) throws IOException {
this.poolId = DeprecatedUTF8.readString(in);
this.blockId = in.readLong();
this.numBytes = in.readLong();
this.generationStamp = in.readLong();
if (numBytes < 0) {
throw new IOException("Unexpected block size: " + numBytes);
}
}
public String getBlockPoolId() {
return poolId;
}
public long getNumBytes() {
return numBytes;
}
public long getBlockId() {
return blockId;
}
public long getGenerationStamp() {
return generationStamp;
}
@Override // Object
public String toString() {
return poolId + ":" + (new org.apache.hadoop.hdfs.protocol.Block(blockId, numBytes, generationStamp));
}
}

View File

@ -1,80 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
@InterfaceAudience.Private
@InterfaceStability.Stable
public class FsPermissionWritable implements Writable {
static final WritableFactory FACTORY = new WritableFactory() {
public Writable newInstance() { return new FsPermissionWritable(); }
};
static { // register a ctor
WritableFactories.setFactory(FsPermissionWritable.class, FACTORY);
}
//POSIX permission style
private short thePermissions = 0;
public static FsPermissionWritable convertPermission(org.apache.hadoop.fs.permission.FsPermission p) {
if (p == null) return null;
return new FsPermissionWritable(p.toShort());
}
public static org.apache.hadoop.fs.permission.FsPermission convertPermission(FsPermissionWritable p) {
if (p == null) return null;
return new org.apache.hadoop.fs.permission.FsPermission(p.thePermissions);
}
public static FsPermissionWritable getDefault() {
return new FsPermissionWritable((short)00777);
}
FsPermissionWritable() {
}
FsPermissionWritable(short p) {
thePermissions = p;
}
@Override
public void write(DataOutput out) throws IOException {
out.writeShort(thePermissions);
}
@Override
public void readFields(DataInput in) throws IOException {
thePermissions = in.readShort();
}
/**
* Create and initialize a {@link FsPermissionWritable} from {@link DataInput}.
*/
public static FsPermissionWritable read(DataInput in) throws IOException {
FsPermissionWritable p = new FsPermissionWritable();
p.readFields(in);
return p;
}
}

View File

@ -1,122 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
/****************************************************
* Provides server default configuration values to clients.
*
****************************************************/
@InterfaceAudience.Public
@InterfaceStability.Stable
public class FsServerDefaultsWritable implements Writable {
static { // register a ctor
WritableFactories.setFactory(FsServerDefaultsWritable.class, new WritableFactory() {
public Writable newInstance() {
return new FsServerDefaultsWritable();
}
});
}
private long blockSize;
private int bytesPerChecksum;
private int writePacketSize;
private short replication;
private int fileBufferSize;
public static org.apache.hadoop.fs.FsServerDefaults convert(
FsServerDefaultsWritable fs) {
if (fs == null) return null;
return new org.apache.hadoop.fs.FsServerDefaults(
fs.getBlockSize(), fs.getBytesPerChecksum(),
fs.getWritePacketSize(), fs.getReplication(), fs.getFileBufferSize());
}
public static FsServerDefaultsWritable convert(
org.apache.hadoop.fs.FsServerDefaults fs) {
if (fs == null) return null;
return new FsServerDefaultsWritable(
fs.getBlockSize(), fs.getBytesPerChecksum(),
fs.getWritePacketSize(), fs.getReplication(), fs.getFileBufferSize());
}
public FsServerDefaultsWritable() {
}
public FsServerDefaultsWritable(long blockSize, int bytesPerChecksum,
int writePacketSize, short replication, int fileBufferSize) {
this.blockSize = blockSize;
this.bytesPerChecksum = bytesPerChecksum;
this.writePacketSize = writePacketSize;
this.replication = replication;
this.fileBufferSize = fileBufferSize;
}
public long getBlockSize() {
return blockSize;
}
public int getBytesPerChecksum() {
return bytesPerChecksum;
}
public int getWritePacketSize() {
return writePacketSize;
}
public short getReplication() {
return replication;
}
public int getFileBufferSize() {
return fileBufferSize;
}
// /////////////////////////////////////////
// Writable
// /////////////////////////////////////////
@Override
@InterfaceAudience.Private
public void write(DataOutput out) throws IOException {
out.writeLong(blockSize);
out.writeInt(bytesPerChecksum);
out.writeInt(writePacketSize);
out.writeShort(replication);
out.writeInt(fileBufferSize);
}
@Override
@InterfaceAudience.Private
public void readFields(DataInput in) throws IOException {
blockSize = in.readLong();
bytesPerChecksum = in.readInt();
writePacketSize = in.readInt();
replication = in.readShort();
fileBufferSize = in.readInt();
}
}

View File

@ -1,348 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
/** Interface that represents the over the wire information for a file.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class HdfsFileStatusWritable implements Writable {
static { // register a ctor
WritableFactories.setFactory
(HdfsFileStatusWritable.class,
new WritableFactory() {
public Writable newInstance() { return new HdfsFileStatusWritable(); }
});
}
private byte[] path; // local name of the inode that's encoded in java UTF8
private byte[] symlink; // symlink target encoded in java UTF8 or null
private long length;
private boolean isdir;
private short block_replication;
private long blocksize;
private long modification_time;
private long access_time;
private FsPermissionWritable permission;
private String owner;
private String group;
public static final byte[] EMPTY_NAME = new byte[0];
static public org.apache.hadoop.hdfs.protocol.HdfsFileStatus
convertHdfsFileStatus(HdfsFileStatusWritable fs) {
if (fs == null) return null;
return new org.apache.hadoop.hdfs.protocol.HdfsFileStatus(fs.getLen(),
fs.isDir(), fs.getReplication(), fs.getBlockSize(),
fs.getModificationTime(), fs.getAccessTime(),
FsPermissionWritable.convertPermission(fs.getPermission()),
fs.getOwner(), fs.getGroup(), fs.getSymlinkInBytes(),
fs.getLocalNameInBytes());
}
static public HdfsFileStatusWritable[] convertHdfsFileStatus(org.apache.hadoop.hdfs.protocol.HdfsFileStatus[] fs) {
if (fs == null) return null;
final int len = fs.length;
HdfsFileStatusWritable[] result = new HdfsFileStatusWritable[len];
for (int i = 0; i < len; ++i) {
if (fs[i] instanceof org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus) {
result[i] =
HdfsLocatedFileStatusWritable.convertLocatedHdfsFileStatus(
(org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus)fs[i]);
} else {
result[i] = HdfsFileStatusWritable.convertHdfsFileStatus(fs[i]);
}
}
return result;
}
public static org.apache.hadoop.hdfs.protocol.HdfsFileStatus[] convertHdfsFileStatus(
HdfsFileStatusWritable[] fs) {
if (fs == null) return null;
final int len = fs.length;
org.apache.hadoop.hdfs.protocol.HdfsFileStatus[] result =
new org.apache.hadoop.hdfs.protocol.HdfsFileStatus[len];
for (int i = 0; i < len; ++i) {
if (fs[i] instanceof HdfsLocatedFileStatusWritable) {
result[i] =
HdfsLocatedFileStatusWritable.convertLocatedHdfsFileStatus((HdfsLocatedFileStatusWritable)fs[i]);
} else {
result[i] = convertHdfsFileStatus(fs[i]);
}
}
return result;
}
public static HdfsFileStatusWritable convertHdfsFileStatus(org.apache.hadoop.hdfs.protocol.HdfsFileStatus fs) {
if (fs == null) return null;
return new HdfsFileStatusWritable(fs.getLen(), fs.isDir(), fs.getReplication(),
fs.getBlockSize(), fs.getModificationTime(), fs.getAccessTime(),
org.apache.hadoop.hdfs.protocolR23Compatible.FsPermissionWritable.
convertPermission(fs.getPermission()),
fs.getOwner(), fs.getGroup(),
fs.getSymlinkInBytes(), fs.getLocalNameInBytes());
}
/**
* default constructor
*/
public HdfsFileStatusWritable() {
this(0, false, 0, 0, 0, 0, null, null, null, null, null);
}
/**
* Constructor
* @param length the number of bytes the file has
* @param isdir if the path is a directory
* @param block_replication the replication factor
* @param blocksize the block size
* @param modification_time modification time
* @param access_time access time
* @param permission permission
* @param owner the owner of the path
* @param group the group of the path
* @param path the local name in java UTF8 encoding the same as that in-memory
*/
public HdfsFileStatusWritable(long length, boolean isdir, int block_replication,
long blocksize, long modification_time, long access_time,
FsPermissionWritable permission, String owner, String group,
byte[] symlink, byte[] path) {
this.length = length;
this.isdir = isdir;
this.block_replication = (short)block_replication;
this.blocksize = blocksize;
this.modification_time = modification_time;
this.access_time = access_time;
this.permission = (permission == null) ? FsPermissionWritable.getDefault() : permission;
this.owner = (owner == null) ? "" : owner;
this.group = (group == null) ? "" : group;
this.symlink = symlink;
this.path = path;
}
/**
* Get the length of this file, in bytes.
* @return the length of this file, in bytes.
*/
final public long getLen() {
return length;
}
/**
* Is this a directory?
* @return true if this is a directory
*/
final public boolean isDir() {
return isdir;
}
/**
* Is this a symbolic link?
* @return true if this is a symbolic link
*/
public boolean isSymlink() {
return symlink != null;
}
/**
* Get the block size of the file.
* @return the number of bytes
*/
final public long getBlockSize() {
return blocksize;
}
/**
* Get the replication factor of a file.
* @return the replication factor of a file.
*/
final public short getReplication() {
return block_replication;
}
/**
* Get the modification time of the file.
* @return the modification time of file in milliseconds since January 1, 1970 UTC.
*/
final public long getModificationTime() {
return modification_time;
}
/**
* Get the access time of the file.
* @return the access time of file in milliseconds since January 1, 1970 UTC.
*/
final public long getAccessTime() {
return access_time;
}
/**
* Get FsPermission associated with the file.
* @return permssion
*/
final public FsPermissionWritable getPermission() {
return permission;
}
/**
* Get the owner of the file.
* @return owner of the file
*/
final public String getOwner() {
return owner;
}
/**
* Get the group associated with the file.
* @return group for the file.
*/
final public String getGroup() {
return group;
}
/**
* Check if the local name is empty
* @return true if the name is empty
*/
final public boolean isEmptyLocalName() {
return path.length == 0;
}
/**
* Get the string representation of the local name
* @return the local name in string
*/
final public String getLocalName() {
return DFSUtil.bytes2String(path);
}
/**
* Get the Java UTF8 representation of the local name
* @return the local name in java UTF8
*/
final public byte[] getLocalNameInBytes() {
return path;
}
/**
* Get the string representation of the full path name
* @param parent the parent path
* @return the full path in string
*/
final public String getFullName(final String parent) {
if (isEmptyLocalName()) {
return parent;
}
StringBuilder fullName = new StringBuilder(parent);
if (!parent.endsWith(Path.SEPARATOR)) {
fullName.append(Path.SEPARATOR);
}
fullName.append(getLocalName());
return fullName.toString();
}
/**
* Get the full path
* @param parent the parent path
* @return the full path
*/
final public Path getFullPath(final Path parent) {
if (isEmptyLocalName()) {
return parent;
}
return new Path(parent, getLocalName());
}
/**
* Get the string representation of the symlink.
* @return the symlink as a string.
*/
final public String getSymlink() {
return DFSUtil.bytes2String(symlink);
}
final public byte[] getSymlinkInBytes() {
return symlink;
}
//////////////////////////////////////////////////
// Writable
//////////////////////////////////////////////////
@Override
public void write(DataOutput out) throws IOException {
out.writeInt(path.length);
out.write(path);
out.writeLong(length);
out.writeBoolean(isdir);
out.writeShort(block_replication);
out.writeLong(blocksize);
out.writeLong(modification_time);
out.writeLong(access_time);
permission.write(out);
Text.writeString(out, owner);
Text.writeString(out, group);
out.writeBoolean(isSymlink());
if (isSymlink()) {
out.writeInt(symlink.length);
out.write(symlink);
}
}
@Override
public void readFields(DataInput in) throws IOException {
int numOfBytes = in.readInt();
if (numOfBytes == 0) {
this.path = EMPTY_NAME;
} else {
this.path = new byte[numOfBytes];
in.readFully(path);
}
this.length = in.readLong();
this.isdir = in.readBoolean();
this.block_replication = in.readShort();
blocksize = in.readLong();
modification_time = in.readLong();
access_time = in.readLong();
permission.readFields(in);
owner = Text.readString(in);
group = Text.readString(in);
if (in.readBoolean()) {
numOfBytes = in.readInt();
this.symlink = new byte[numOfBytes];
in.readFully(symlink);
}
}
}

View File

@ -1,116 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Interface that represents the over the wire information
* including block locations for a file.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class HdfsLocatedFileStatusWritable extends HdfsFileStatusWritable {
private LocatedBlocksWritable locations;
/**
* Default constructor
*/
public HdfsLocatedFileStatusWritable() {
}
/**
* Constructor
*
* @param length size
* @param isdir if this is directory
* @param block_replication the file's replication factor
* @param blocksize the file's block size
* @param modification_time most recent modification time
* @param access_time most recent access time
* @param permission permission
* @param owner owner
* @param group group
* @param symlink symbolic link
* @param path local path name in java UTF8 format
* @param locations block locations
*/
public HdfsLocatedFileStatusWritable(long length, boolean isdir,
int block_replication,
long blocksize, long modification_time, long access_time,
FsPermissionWritable permission, String owner, String group,
byte[] symlink, byte[] path, LocatedBlocksWritable locations) {
super(length, isdir, block_replication, blocksize, modification_time,
access_time, permission, owner, group, symlink, path);
this.locations = locations;
}
static public org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus
convertLocatedHdfsFileStatus(HdfsLocatedFileStatusWritable fs) {
if (fs == null) return null;
return new org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus(fs.getLen(),
fs.isDir(), fs.getReplication(), fs.getBlockSize(),
fs.getModificationTime(), fs.getAccessTime(),
FsPermissionWritable.convertPermission(fs.getPermission()),
fs.getOwner(), fs.getGroup(), fs.getSymlinkInBytes(),
fs.getLocalNameInBytes(),
LocatedBlocksWritable.convertLocatedBlocks(fs.getBlockLocations()));
}
static public HdfsLocatedFileStatusWritable convertLocatedHdfsFileStatus(
org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus fs) {
if (fs == null) return null;
return new HdfsLocatedFileStatusWritable(fs.getLen(),
fs.isDir(), fs.getReplication(), fs.getBlockSize(),
fs.getModificationTime(), fs.getAccessTime(),
org.apache.hadoop.hdfs.protocolR23Compatible.FsPermissionWritable.
convertPermission(fs.getPermission()),
fs.getOwner(), fs.getGroup(), fs.getSymlinkInBytes(),
fs.getLocalNameInBytes(),
LocatedBlocksWritable.convertLocatedBlocks(fs.getBlockLocations()));
}
public LocatedBlocksWritable getBlockLocations() {
return locations;
}
//////////////////////////////////////////////////
// Writable
//////////////////////////////////////////////////
@Override
public void write(DataOutput out) throws IOException {
super.write(out);
if (!isDir() && !isSymlink()) {
locations.write(out);
}
}
@Override
public void readFields(DataInput in) throws IOException {
super.readFields(in);
if (!isDir() && !isSymlink()) {
locations = new LocatedBlocksWritable();
locations.readFields(in);
}
}
}

View File

@ -1,114 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
/**
* This class is used on the server side. Calls come across the wire for the
* protocol family of Release 23 onwards. This class translates the R23 data
* types to the native data types used inside the NN as specified in the generic
* JournalProtocol.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class JournalProtocolServerSideTranslatorR23 implements
JournalWireProtocol {
final private JournalProtocol server;
/**
* Constructor
*
* @param server - the NN server
* @throws IOException
*/
public JournalProtocolServerSideTranslatorR23(JournalProtocol server)
throws IOException {
this.server = server;
}
/**
* The client side will redirect getProtocolSignature to
* getProtocolSignature2.
*
* However the RPC layer below on the Server side will call getProtocolVersion
* and possibly in the future getProtocolSignature. Hence we still implement
* it even though the end client's call will never reach here.
*/
@Override
public ProtocolSignature getProtocolSignature(String protocol,
long clientVersion, int clientMethodsHash) throws IOException {
/**
* Don't forward this to the server. The protocol version and signature is
* that of {@link JournalProtocol}
*
*/
if (!protocol.equals(RPC.getProtocolName(JournalWireProtocol.class))) {
throw new IOException("Namenode Serverside implements " +
RPC.getProtocolName(JournalWireProtocol.class) +
". The following requested protocol is unknown: " + protocol);
}
return ProtocolSignature.getProtocolSignature(clientMethodsHash,
JournalWireProtocol.versionID, JournalWireProtocol.class);
}
@Override
public ProtocolSignatureWritable
getProtocolSignature2(
String protocol, long clientVersion, int clientMethodsHash)
throws IOException {
/**
* Don't forward this to the server. The protocol version and signature is
* that of {@link ClientNamenodeProtocol}
*
*/
return ProtocolSignatureWritable.convert(
this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
}
@Override
public long getProtocolVersion(String protocol, long clientVersion)
throws IOException {
if (protocol.equals(RPC.getProtocolName(JournalWireProtocol.class))) {
return JournalWireProtocol.versionID;
}
throw new IOException("Namenode Serverside implements " +
RPC.getProtocolName(JournalWireProtocol.class) +
". The following requested protocol is unknown: " + protocol);
}
@Override
public void journal(NamenodeRegistrationWritable registration,
long firstTxnId, int numTxns, byte[] records) throws IOException {
server.journal(registration.convert(), firstTxnId, numTxns, records);
}
@Override
public void startLogSegment(NamenodeRegistrationWritable registration,
long txid) throws IOException {
server.startLogSegment(registration.convert(), txid);
}
}

View File

@ -1,80 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.Closeable;
import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
/**
* This class forwards NN's ClientProtocol calls as RPC calls to the NN server
* while translating from the parameter types used in ClientProtocol to those
* used in protocolR23Compatile.*.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class JournalProtocolTranslatorR23 implements
JournalProtocol, Closeable {
private final JournalWireProtocol rpcProxy;
public JournalProtocolTranslatorR23(InetSocketAddress nameNodeAddr,
Configuration conf) throws IOException {
rpcProxy = RPC.getProxy(JournalWireProtocol.class,
JournalWireProtocol.versionID, nameNodeAddr, conf);
}
@Override
public void close() {
RPC.stopProxy(rpcProxy);
}
@Override
public long getProtocolVersion(String protocolName, long clientVersion)
throws IOException {
return rpcProxy.getProtocolVersion(protocolName, clientVersion);
}
@Override
public ProtocolSignature getProtocolSignature(String protocol,
long clientVersion, int clientMethodsHash) throws IOException {
return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
protocol, clientVersion, clientMethodsHash));
}
@Override
public void journal(NamenodeRegistration registration, long firstTxnId,
int numTxns, byte[] records) throws IOException {
rpcProxy.journal(NamenodeRegistrationWritable.convert(registration),
firstTxnId, numTxns, records);
}
@Override
public void startLogSegment(NamenodeRegistration registration, long txid)
throws IOException {
rpcProxy.startLogSegment(NamenodeRegistrationWritable.convert(registration),
txid);
}
}

View File

@ -1,81 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.security.KerberosInfo;
/**
* This class defines the actual protocol used to communicate with the
* NN via RPC using writable types.
* The parameters in the methods which are specified in the
* package are separate from those used internally in the NN and DFSClient
* and hence need to be converted using {@link JournalProtocolTranslatorR23}
* and {@link JournalProtocolServerSideTranslatorR23}.
*
*/
@KerberosInfo(
serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
@InterfaceAudience.Private
public interface JournalWireProtocol extends VersionedProtocol {
/**
* The rules for changing this protocol are the same as that for
* {@link ClientNamenodeWireProtocol} - see that java file for details.
*/
public static final long versionID = 1L;
/**
* Journal edit records.
* This message is sent by the active name-node to the backup node
* via {@code EditLogBackupOutputStream} in order to synchronize meta-data
* changes with the backup namespace image.
*
* @param registration active node registration
* @param firstTxnId the first transaction of this batch
* @param numTxns number of transactions
* @param records byte array containing serialized journal records
*/
public void journal(NamenodeRegistrationWritable registration,
long firstTxnId,
int numTxns,
byte[] records) throws IOException;
/**
* Notify the BackupNode that the NameNode has rolled its edit logs
* and is now writing a new log segment.
* @param registration the registration of the active NameNode
* @param txid the first txid in the new log
*/
public void startLogSegment(NamenodeRegistrationWritable registration,
long txid) throws IOException;
/**
* This method is defined to get the protocol signature using
* the R23 protocol - hence we have added the suffix of 2 the method name
* to avoid conflict.
*/
public org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable
getProtocolSignature2(String protocol, long clientVersion,
int clientMethodsHash) throws IOException;
}

View File

@ -1,253 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.io.*;
import java.io.*;
import java.util.ArrayList;
import java.util.List;
/****************************************************
* A LocatedBlock is a pair of Block, DatanodeInfo[]
* objects. It tells where to find a Block.
*
****************************************************/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class LocatedBlockWritable implements Writable {
static { // register a ctor
WritableFactories.setFactory
(LocatedBlockWritable.class,
new WritableFactory() {
public Writable newInstance() { return new LocatedBlockWritable(); }
});
}
private ExtendedBlockWritable b;
private long offset; // offset of the first byte of the block in the file
private DatanodeInfoWritable[] locs;
// corrupt flag is true if all of the replicas of a block are corrupt.
// else false. If block has few corrupt replicas, they are filtered and
// their locations are not part of this object
private boolean corrupt;
private TokenWritable blockToken = new TokenWritable();
static public org.apache.hadoop.hdfs.protocol.LocatedBlock
convertLocatedBlock(LocatedBlockWritable lb) {
if (lb == null) return null;
org.apache.hadoop.hdfs.protocol.LocatedBlock result =
new org.apache.hadoop.hdfs.protocol.LocatedBlock(ExtendedBlockWritable.
convertExtendedBlock(lb.getBlock()),
DatanodeInfoWritable.convertDatanodeInfo(
lb.getLocations()), lb.getStartOffset(), lb.isCorrupt());
// Fill in the token
TokenWritable tok = lb.getBlockToken();
result.setBlockToken(
new org.apache.hadoop.security.token.Token<BlockTokenIdentifier>(
tok.getIdentifier(), tok.getPassword(), tok.getKind(),
tok.getService()));
return result;
}
public static LocatedBlockWritable
convertLocatedBlock(org.apache.hadoop.hdfs.protocol.LocatedBlock lb) {
if (lb == null) return null;
LocatedBlockWritable result =
new LocatedBlockWritable(ExtendedBlockWritable.convertExtendedBlock(lb.getBlock()),
DatanodeInfoWritable.convertDatanodeInfo(lb.getLocations()),
lb.getStartOffset(), lb.isCorrupt());
// Fill in the token
org.apache.hadoop.security.token.Token<BlockTokenIdentifier> tok =
lb.getBlockToken();
result.setBlockToken(new TokenWritable(tok.getIdentifier(), tok.getPassword(),
tok.getKind(), tok.getService()));
return result;
}
static public LocatedBlockWritable[]
convertLocatedBlock(org.apache.hadoop.hdfs.protocol.LocatedBlock[] lb) {
if (lb == null) return null;
final int len = lb.length;
LocatedBlockWritable[] result = new LocatedBlockWritable[len];
for (int i = 0; i < len; ++i) {
result[i] = new LocatedBlockWritable(
ExtendedBlockWritable.convertExtendedBlock(lb[i].getBlock()),
DatanodeInfoWritable.convertDatanodeInfo(lb[i].getLocations()),
lb[i].getStartOffset(), lb[i].isCorrupt());
}
return result;
}
static public org.apache.hadoop.hdfs.protocol.LocatedBlock[]
convertLocatedBlock(LocatedBlockWritable[] lb) {
if (lb == null) return null;
final int len = lb.length;
org.apache.hadoop.hdfs.protocol.LocatedBlock[] result =
new org.apache.hadoop.hdfs.protocol.LocatedBlock[len];
for (int i = 0; i < len; ++i) {
result[i] = new org.apache.hadoop.hdfs.protocol.LocatedBlock(
ExtendedBlockWritable.convertExtendedBlock(lb[i].getBlock()),
DatanodeInfoWritable.convertDatanodeInfo(lb[i].getLocations()),
lb[i].getStartOffset(), lb[i].isCorrupt());
}
return result;
}
static public List<org.apache.hadoop.hdfs.protocol.LocatedBlock>
convertLocatedBlock(
List<org.apache.hadoop.hdfs.protocolR23Compatible.LocatedBlockWritable> lb) {
if (lb == null) return null;
final int len = lb.size();
List<org.apache.hadoop.hdfs.protocol.LocatedBlock> result =
new ArrayList<org.apache.hadoop.hdfs.protocol.LocatedBlock>(len);
for (int i = 0; i < len; ++i) {
result.add(LocatedBlockWritable.convertLocatedBlock(lb.get(i)));
}
return result;
}
static public List<LocatedBlockWritable>
convertLocatedBlock2(List<org.apache.hadoop.hdfs.protocol.LocatedBlock> lb) {
if (lb == null) return null;
final int len = lb.size();
List<LocatedBlockWritable> result = new ArrayList<LocatedBlockWritable>(len);
for (int i = 0; i < len; ++i) {
result.add(LocatedBlockWritable.convertLocatedBlock(lb.get(i)));
}
return result;
}
public LocatedBlockWritable() {
this(new ExtendedBlockWritable(), new DatanodeInfoWritable[0], 0L, false);
}
public LocatedBlockWritable(ExtendedBlockWritable eb) {
this(eb, new DatanodeInfoWritable[0], 0L, false);
}
public LocatedBlockWritable(ExtendedBlockWritable b, DatanodeInfoWritable[] locs) {
this(b, locs, -1, false); // startOffset is unknown
}
public LocatedBlockWritable(ExtendedBlockWritable b, DatanodeInfoWritable[] locs, long startOffset) {
this(b, locs, startOffset, false);
}
public LocatedBlockWritable(ExtendedBlockWritable b, DatanodeInfoWritable[] locs, long startOffset,
boolean corrupt) {
this.b = b;
this.offset = startOffset;
this.corrupt = corrupt;
if (locs==null) {
this.locs = new DatanodeInfoWritable[0];
} else {
this.locs = locs;
}
}
public TokenWritable getBlockToken() {
return blockToken;
}
public void setBlockToken(TokenWritable token) {
this.blockToken = token;
}
public ExtendedBlockWritable getBlock() {
return b;
}
public DatanodeInfoWritable[] getLocations() {
return locs;
}
public long getStartOffset() {
return offset;
}
public long getBlockSize() {
return b.getNumBytes();
}
void setStartOffset(long value) {
this.offset = value;
}
void setCorrupt(boolean corrupt) {
this.corrupt = corrupt;
}
public boolean isCorrupt() {
return this.corrupt;
}
///////////////////////////////////////////
// Writable
///////////////////////////////////////////
@Override
public void write(DataOutput out) throws IOException {
blockToken.write(out);
out.writeBoolean(corrupt);
out.writeLong(offset);
b.write(out);
out.writeInt(locs.length);
for (int i = 0; i < locs.length; i++) {
locs[i].write(out);
}
}
@Override
public void readFields(DataInput in) throws IOException {
blockToken.readFields(in);
this.corrupt = in.readBoolean();
offset = in.readLong();
this.b = new ExtendedBlockWritable();
b.readFields(in);
int count = in.readInt();
this.locs = new DatanodeInfoWritable[count];
for (int i = 0; i < locs.length; i++) {
locs[i] = new DatanodeInfoWritable();
locs[i].readFields(in);
}
}
/** Read LocatedBlock from in. */
public static LocatedBlockWritable read(DataInput in) throws IOException {
final LocatedBlockWritable lb = new LocatedBlockWritable();
lb.readFields(in);
return lb;
}
@Override
public String toString() {
return getClass().getSimpleName() + "{" + b
+ "; getBlockSize()=" + getBlockSize()
+ "; corrupt=" + corrupt
+ "; offset=" + offset
+ "; locs=" + java.util.Arrays.asList(locs)
+ "}";
}
}

View File

@ -1,197 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.List;
import java.util.ArrayList;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
/**
* Collection of blocks with their locations and the file length.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class LocatedBlocksWritable implements Writable {
private long fileLength;
private List<LocatedBlockWritable> blocks; // array of blocks with prioritized locations
private boolean underConstruction;
private LocatedBlockWritable lastLocatedBlock = null;
private boolean isLastBlockComplete = false;
public static org.apache.hadoop.hdfs.protocol.LocatedBlocks convertLocatedBlocks(
LocatedBlocksWritable lb) {
if (lb == null) {
return null;
}
return new org.apache.hadoop.hdfs.protocol.LocatedBlocks(
lb.getFileLength(), lb.isUnderConstruction(),
LocatedBlockWritable.convertLocatedBlock(lb.getLocatedBlocks()),
LocatedBlockWritable.convertLocatedBlock(lb.getLastLocatedBlock()),
lb.isLastBlockComplete());
}
public static LocatedBlocksWritable convertLocatedBlocks(
org.apache.hadoop.hdfs.protocol.LocatedBlocks lb) {
if (lb == null) {
return null;
}
return new LocatedBlocksWritable(lb.getFileLength(), lb.isUnderConstruction(),
LocatedBlockWritable.convertLocatedBlock2(lb.getLocatedBlocks()),
LocatedBlockWritable.convertLocatedBlock(lb.getLastLocatedBlock()),
lb.isLastBlockComplete());
}
public LocatedBlocksWritable() {
this(0, false, null, null, false);
}
/** public Constructor */
public LocatedBlocksWritable(long flength, boolean isUnderConstuction,
List<LocatedBlockWritable> blks,
LocatedBlockWritable lastBlock, boolean isLastBlockCompleted) {
fileLength = flength;
blocks = blks;
underConstruction = isUnderConstuction;
this.lastLocatedBlock = lastBlock;
this.isLastBlockComplete = isLastBlockCompleted;
}
/**
* Get located blocks.
*/
public List<LocatedBlockWritable> getLocatedBlocks() {
return blocks;
}
/** Get the last located block. */
public LocatedBlockWritable getLastLocatedBlock() {
return lastLocatedBlock;
}
/** Is the last block completed? */
public boolean isLastBlockComplete() {
return isLastBlockComplete;
}
/**
* Get located block.
*/
public LocatedBlockWritable get(int index) {
return blocks.get(index);
}
/**
* Get number of located blocks.
*/
public int locatedBlockCount() {
return blocks == null ? 0 : blocks.size();
}
/**
* Get file length
*/
public long getFileLength() {
return this.fileLength;
}
/**
* Return ture if file was under construction when
* this LocatedBlocks was constructed, false otherwise.
*/
public boolean isUnderConstruction() {
return underConstruction;
}
//////////////////////////////////////////////////
// Writable
//////////////////////////////////////////////////
static { // register a ctor
WritableFactories.setFactory
(LocatedBlocksWritable.class,
new WritableFactory() {
public Writable newInstance() { return new LocatedBlocksWritable(); }
});
}
@Override
public void write(DataOutput out) throws IOException {
out.writeLong(this.fileLength);
out.writeBoolean(underConstruction);
//write the last located block
final boolean isNull = lastLocatedBlock == null;
out.writeBoolean(isNull);
if (!isNull) {
lastLocatedBlock.write(out);
}
out.writeBoolean(isLastBlockComplete);
// write located blocks
int nrBlocks = locatedBlockCount();
out.writeInt(nrBlocks);
if (nrBlocks == 0) {
return;
}
for (LocatedBlockWritable blk : this.blocks) {
blk.write(out);
}
}
@Override
public void readFields(DataInput in) throws IOException {
this.fileLength = in.readLong();
underConstruction = in.readBoolean();
//read the last located block
final boolean isNull = in.readBoolean();
if (!isNull) {
lastLocatedBlock = LocatedBlockWritable.read(in);
}
isLastBlockComplete = in.readBoolean();
// read located blocks
int nrBlocks = in.readInt();
this.blocks = new ArrayList<LocatedBlockWritable>(nrBlocks);
for (int idx = 0; idx < nrBlocks; idx++) {
LocatedBlockWritable blk = new LocatedBlockWritable();
blk.readFields(in);
this.blocks.add(blk);
}
}
@Override
public String toString() {
final StringBuilder b = new StringBuilder(getClass().getSimpleName());
b.append("{")
.append("\n fileLength=").append(fileLength)
.append("\n underConstruction=").append(underConstruction)
.append("\n blocks=").append(blocks)
.append("\n lastLocatedBlock=").append(lastLocatedBlock)
.append("\n isLastBlockComplete=").append(isLastBlockComplete)
.append("}");
return b.toString();
}
}

View File

@ -1,70 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
/**
* Base class for name-node command.
* Issued by the name-node to notify other name-nodes what should be done.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class NamenodeCommandWritable implements Writable {
private int action;
static {
WritableFactories.setFactory(NamenodeCommandWritable.class,
new WritableFactory() {
public Writable newInstance() {return new NamenodeCommandWritable();}
});
}
public NamenodeCommandWritable() {
}
public NamenodeCommandWritable(int action) {
this.action = action;
}
@Override
public void write(DataOutput out) throws IOException {
out.writeInt(this.action);
}
@Override
public void readFields(DataInput in) throws IOException {
this.action = in.readInt();
}
public static NamenodeCommandWritable convert(NamenodeCommand cmd) {
return new NamenodeCommandWritable(cmd.getAction());
}
public NamenodeCommand convert() {
return new NamenodeCommand(action);
}
}

View File

@ -1,163 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
/**
* This class is used on the server side.
* Calls come across the wire for the protocol family of Release 23 onwards.
* This class translates the R23 data types to the internal data types used
* inside the DN as specified in the generic NamenodeProtocol.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class NamenodeProtocolServerSideTranslatorR23 implements
NamenodeWireProtocol {
final private NamenodeProtocol server;
/**
* @param server - the NN server
* @throws IOException
*/
public NamenodeProtocolServerSideTranslatorR23(
NamenodeProtocol server) throws IOException {
this.server = server;
}
/**
* the client side will redirect getProtocolSignature to
* getProtocolSignature2.
*
* However the RPC layer below on the Server side will call
* getProtocolVersion and possibly in the future getProtocolSignature.
* Hence we still implement it even though the end client's call will
* never reach here.
*/
@Override
public ProtocolSignature getProtocolSignature(String protocol,
long clientVersion, int clientMethodsHash) throws IOException {
/**
* Don't forward this to the server. The protocol version and
* signature is that of {@link NamenodeProtocol}
*/
if (!protocol.equals(RPC.getProtocolName(
NamenodeWireProtocol.class))) {
throw new IOException("Namenode Serverside implements " +
NamenodeWireProtocol.class +
". The following requested protocol is unknown: " + protocol);
}
return ProtocolSignature.getProtocolSignature(clientMethodsHash,
NamenodeWireProtocol.versionID,
NamenodeWireProtocol.class);
}
@Override
public ProtocolSignatureWritable getProtocolSignature2(String protocol,
long clientVersion, int clientMethodsHash) throws IOException {
/**
* Don't forward this to the server. The protocol version and
* signature is that of {@link ClientNamenodeProtocol}
*/
return ProtocolSignatureWritable.convert(
this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
}
@Override
public long getProtocolVersion(String protocol, long clientVersion)
throws IOException {
if (protocol.equals(RPC.getProtocolName(
NamenodeWireProtocol.class))) {
return NamenodeWireProtocol.versionID;
}
throw new IOException("Datanode Serverside implements " +
NamenodeWireProtocol.class +
". The following requested protocol is unknown: " + protocol);
}
@Override
public BlocksWithLocationsWritable getBlocks(DatanodeInfoWritable datanode,
long size) throws IOException {
BlocksWithLocations locs = server.getBlocks(
DatanodeInfoWritable.convertDatanodeInfo(datanode), size);
return BlocksWithLocationsWritable.convert(locs);
}
@Override
public ExportedBlockKeysWritable getBlockKeys() throws IOException {
return ExportedBlockKeysWritable.convert(server.getBlockKeys());
}
@Override
public long getTransactionID() throws IOException {
return server.getTransactionID();
}
@Override
@SuppressWarnings("deprecation")
public CheckpointSignatureWritable rollEditLog() throws IOException {
return CheckpointSignatureWritable.convert(server.rollEditLog());
}
@Override
public NamespaceInfoWritable versionRequest() throws IOException {
return NamespaceInfoWritable.convert(server.versionRequest());
}
@Override
public void errorReport(NamenodeRegistrationWritable registration,
int errorCode, String msg) throws IOException {
server.errorReport(registration.convert(), errorCode, msg);
}
@Override
public NamenodeRegistrationWritable register(
NamenodeRegistrationWritable registration) throws IOException {
return NamenodeRegistrationWritable.convert(server
.register(registration.convert()));
}
@Override
public NamenodeCommandWritable startCheckpoint(
NamenodeRegistrationWritable registration) throws IOException {
return NamenodeCommandWritable.convert(server.startCheckpoint(registration
.convert()));
}
@Override
public void endCheckpoint(NamenodeRegistrationWritable registration,
CheckpointSignatureWritable sig) throws IOException {
server.endCheckpoint(registration.convert(), sig.convert());
}
@Override
public RemoteEditLogManifestWritable getEditLogManifest(long sinceTxId)
throws IOException {
return RemoteEditLogManifestWritable.convert(server
.getEditLogManifest(sinceTxId));
}
}

View File

@ -1,180 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.Closeable;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
import org.apache.hadoop.io.retry.RetryPolicies;
import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.io.retry.RetryProxy;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
/**
* This class forwards NN's NamenodeProtocol calls as RPC calls to the NN server
* while translating from the parameter types used in NamenodeProtocol to those
* used in protocolR23Compatile.*.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class NamenodeProtocolTranslatorR23 implements
NamenodeProtocol, Closeable {
final private NamenodeWireProtocol rpcProxy;
private static NamenodeWireProtocol createNamenode(
InetSocketAddress nameNodeAddr, Configuration conf,
UserGroupInformation ugi) throws IOException {
return RPC.getProxy(NamenodeWireProtocol.class,
NamenodeWireProtocol.versionID, nameNodeAddr, ugi, conf,
NetUtils.getSocketFactory(conf, NamenodeWireProtocol.class));
}
/** Create a {@link NameNode} proxy */
static NamenodeWireProtocol createNamenodeWithRetry(
NamenodeWireProtocol rpcNamenode) {
RetryPolicy createPolicy = RetryPolicies
.retryUpToMaximumCountWithFixedSleep(5,
HdfsConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
Map<Class<? extends Exception>, RetryPolicy> remoteExceptionToPolicyMap =
new HashMap<Class<? extends Exception>, RetryPolicy>();
remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class,
createPolicy);
Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
new HashMap<Class<? extends Exception>, RetryPolicy>();
exceptionToPolicyMap.put(RemoteException.class, RetryPolicies
.retryByRemoteException(RetryPolicies.TRY_ONCE_THEN_FAIL,
remoteExceptionToPolicyMap));
RetryPolicy methodPolicy = RetryPolicies.retryByException(
RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
Map<String, RetryPolicy> methodNameToPolicyMap = new HashMap<String, RetryPolicy>();
methodNameToPolicyMap.put("create", methodPolicy);
return (NamenodeWireProtocol) RetryProxy.create(
NamenodeWireProtocol.class, rpcNamenode, methodNameToPolicyMap);
}
public NamenodeProtocolTranslatorR23(InetSocketAddress nameNodeAddr,
Configuration conf, UserGroupInformation ugi) throws IOException {
rpcProxy = createNamenodeWithRetry(createNamenode(nameNodeAddr, conf, ugi));
}
public void close() {
RPC.stopProxy(rpcProxy);
}
@Override
public ProtocolSignature getProtocolSignature(String protocolName,
long clientVersion, int clientMethodHash)
throws IOException {
return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
protocolName, clientVersion, clientMethodHash));
}
@Override
public long getProtocolVersion(String protocolName, long clientVersion) throws IOException {
return rpcProxy.getProtocolVersion(protocolName, clientVersion);
}
@Override
public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
throws IOException {
return rpcProxy.getBlocks(
DatanodeInfoWritable.convertDatanodeInfo(datanode), size).convert();
}
@Override
public ExportedBlockKeys getBlockKeys() throws IOException {
return rpcProxy.getBlockKeys().convert();
}
@Override
public long getTransactionID() throws IOException {
return rpcProxy.getTransactionID();
}
@Override
@SuppressWarnings("deprecation")
public CheckpointSignature rollEditLog() throws IOException {
return rpcProxy.rollEditLog().convert();
}
@Override
public NamespaceInfo versionRequest() throws IOException {
return rpcProxy.versionRequest().convert();
}
@Override
public void errorReport(NamenodeRegistration registration, int errorCode,
String msg) throws IOException {
rpcProxy.errorReport(NamenodeRegistrationWritable.convert(registration),
errorCode, msg);
}
@Override
public NamenodeRegistration register(NamenodeRegistration registration)
throws IOException {
return rpcProxy
.register(NamenodeRegistrationWritable.convert(registration)).convert();
}
@Override
public NamenodeCommand startCheckpoint(NamenodeRegistration registration)
throws IOException {
return rpcProxy.startCheckpoint(
NamenodeRegistrationWritable.convert(registration)).convert();
}
@Override
public void endCheckpoint(NamenodeRegistration registration,
CheckpointSignature sig) throws IOException {
rpcProxy.endCheckpoint(NamenodeRegistrationWritable.convert(registration),
CheckpointSignatureWritable.convert(sig));
}
@Override
public RemoteEditLogManifest getEditLogManifest(long sinceTxId)
throws IOException {
return rpcProxy.getEditLogManifest(sinceTxId).convert();
}
}

View File

@ -1,98 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
/**
* Information sent by a subordinate name-node to the active name-node
* during the registration process.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class NamenodeRegistrationWritable implements Writable {
private String rpcAddress; // RPC address of the node
private String httpAddress; // HTTP address of the node
private NamenodeRole role; // node role
private StorageInfoWritable storageInfo;
public NamenodeRegistrationWritable() { }
public NamenodeRegistrationWritable(String address,
String httpAddress,
NamenodeRole role,
StorageInfo storageInfo) {
this.rpcAddress = address;
this.httpAddress = httpAddress;
this.role = role;
this.storageInfo = StorageInfoWritable.convert(storageInfo);
}
/////////////////////////////////////////////////
// Writable
/////////////////////////////////////////////////
static {
WritableFactories.setFactory
(NamenodeRegistrationWritable.class,
new WritableFactory() {
public Writable newInstance() {
return new NamenodeRegistrationWritable();
}
});
}
@Override // Writable
public void write(DataOutput out) throws IOException {
Text.writeString(out, rpcAddress);
Text.writeString(out, httpAddress);
Text.writeString(out, role.name());
storageInfo.write(out);
}
@Override // Writable
public void readFields(DataInput in) throws IOException {
rpcAddress = Text.readString(in);
httpAddress = Text.readString(in);
role = NamenodeRole.valueOf(Text.readString(in));
storageInfo = new StorageInfoWritable();
storageInfo.readFields(in);
}
public static NamenodeRegistrationWritable convert(NamenodeRegistration reg) {
return new NamenodeRegistrationWritable(reg.getAddress(),
reg.getHttpAddress(), reg.getRole(), reg);
}
public NamenodeRegistration convert() {
return new NamenodeRegistration(rpcAddress, httpAddress,
storageInfo.convert(), role);
}
}

View File

@ -1,169 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.server.protocol.CheckpointCommand;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.security.KerberosInfo;
/*****************************************************************************
* Protocol that a secondary NameNode uses to communicate with the NameNode.
* It's used to get part of the name node state
*****************************************************************************/
/**
* This class defines the actual protocol used to communicate between namenodes.
* The parameters in the methods which are specified in the
* package are separate from those used internally in the DN and DFSClient
* and hence need to be converted using {@link NamenodeProtocolTranslatorR23}
* and {@link NamenodeProtocolServerSideTranslatorR23}.
*/
@KerberosInfo(
serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
@InterfaceAudience.Private
public interface NamenodeWireProtocol extends VersionedProtocol {
/**
* The rules for changing this protocol are the same as that for
* {@link ClientNamenodeWireProtocol} - see that java file for details.
*/
public static final long versionID = 6L;
/**
* Get a list of blocks belonging to <code>datanode</code>
* whose total size equals <code>size</code>.
*
* @see org.apache.hadoop.hdfs.server.balancer.Balancer
* @param datanode a data node
* @param size requested size
* @return a list of blocks & their locations
* @throws RemoteException if size is less than or equal to 0 or
* datanode does not exist
*/
public BlocksWithLocationsWritable getBlocks(DatanodeInfoWritable datanode,
long size) throws IOException;
/**
* Get the current block keys
*
* @return ExportedBlockKeys containing current block keys
* @throws IOException
*/
public ExportedBlockKeysWritable getBlockKeys() throws IOException;
/**
* @return The most recent transaction ID that has been synced to
* persistent storage.
* @throws IOException
*/
public long getTransactionID() throws IOException;
/**
* Closes the current edit log and opens a new one. The
* call fails if the file system is in SafeMode.
* @throws IOException
* @return a unique token to identify this transaction.
* @deprecated
* See {@link org.apache.hadoop.hdfs.server.namenode.SecondaryNameNode}
*/
@Deprecated
public CheckpointSignatureWritable rollEditLog() throws IOException;
/**
* Request name-node version and storage information.
* @throws IOException
*/
public NamespaceInfoWritable versionRequest() throws IOException;
/**
* Report to the active name-node an error occurred on a subordinate node.
* Depending on the error code the active node may decide to unregister the
* reporting node.
*
* @param registration requesting node.
* @param errorCode indicates the error
* @param msg free text description of the error
* @throws IOException
*/
public void errorReport(NamenodeRegistrationWritable registration,
int errorCode,
String msg) throws IOException;
/**
* Register a subordinate name-node like backup node.
*
* @return {@link NamenodeRegistration} of the node,
* which this node has just registered with.
*/
public NamenodeRegistrationWritable register(
NamenodeRegistrationWritable registration) throws IOException;
/**
* A request to the active name-node to start a checkpoint.
* The name-node should decide whether to admit it or reject.
* The name-node also decides what should be done with the backup node
* image before and after the checkpoint.
*
* @see CheckpointCommand
* @see NamenodeCommandWritable
* @see #ACT_SHUTDOWN
*
* @param registration the requesting node
* @return {@link CheckpointCommand} if checkpoint is allowed.
* @throws IOException
*/
public NamenodeCommandWritable startCheckpoint(
NamenodeRegistrationWritable registration) throws IOException;
/**
* A request to the active name-node to finalize
* previously started checkpoint.
*
* @param registration the requesting node
* @param sig {@code CheckpointSignature} which identifies the checkpoint.
* @throws IOException
*/
public void endCheckpoint(NamenodeRegistrationWritable registration,
CheckpointSignatureWritable sig) throws IOException;
/**
* Return a structure containing details about all edit logs
* available to be fetched from the NameNode.
* @param sinceTxId return only logs that contain transactions >= sinceTxId
*/
public RemoteEditLogManifestWritable getEditLogManifest(long sinceTxId)
throws IOException;
/**
* This method is defined to get the protocol signature using
* the R23 protocol - hence we have added the suffix of 2 the method name
* to avoid conflict.
*/
public org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable
getProtocolSignature2(String protocol,
long clientVersion,
int clientMethodsHash) throws IOException;
}

View File

@ -1,100 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.DeprecatedUTF8;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
import org.apache.hadoop.io.WritableUtils;
/**
* NamespaceInfoWritable is returned by the name-node in reply
* to a data-node handshake.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class NamespaceInfoWritable extends StorageInfo {
private String buildVersion;
private int distributedUpgradeVersion;
private String blockPoolID = "";
private StorageInfoWritable storageInfo;
public NamespaceInfoWritable() {
super();
buildVersion = null;
}
public NamespaceInfoWritable(int nsID, String clusterID, String bpID,
long cT, int duVersion) {
this.blockPoolID = bpID;
this.buildVersion = Storage.getBuildVersion();
this.distributedUpgradeVersion = duVersion;
storageInfo = new StorageInfoWritable(HdfsConstants.LAYOUT_VERSION, nsID,
clusterID, cT);
}
/////////////////////////////////////////////////
// Writable
/////////////////////////////////////////////////
static { // register a ctor
WritableFactories.setFactory
(NamespaceInfoWritable.class,
new WritableFactory() {
public Writable newInstance() { return new NamespaceInfoWritable(); }
});
}
@Override
public void write(DataOutput out) throws IOException {
DeprecatedUTF8.writeString(out, buildVersion);
storageInfo.write(out);
out.writeInt(distributedUpgradeVersion);
WritableUtils.writeString(out, blockPoolID);
}
@Override
public void readFields(DataInput in) throws IOException {
buildVersion = DeprecatedUTF8.readString(in);
storageInfo.readFields(in);
distributedUpgradeVersion = in.readInt();
blockPoolID = WritableUtils.readString(in);
}
public static NamespaceInfoWritable convert(NamespaceInfo info) {
return new NamespaceInfoWritable(info.getNamespaceID(), info.getClusterID(),
info.getBlockPoolID(), info.getCTime(),
info.getDistributedUpgradeVersion());
}
public NamespaceInfo convert() {
return new NamespaceInfo(namespaceID, clusterID, blockPoolID, cTime,
distributedUpgradeVersion);
}
}

View File

@ -1,90 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
import com.google.common.collect.Lists;
/**
* An enumeration of logs available on a remote NameNode.
*/
public class RemoteEditLogManifestWritable implements Writable {
private List<RemoteEditLogWritable> logs;
static { // register a ctor
WritableFactories.setFactory(RemoteEditLogManifestWritable.class,
new WritableFactory() {
public Writable newInstance() {
return new RemoteEditLogManifestWritable();
}
});
}
public RemoteEditLogManifestWritable() {
}
public RemoteEditLogManifestWritable(List<RemoteEditLogWritable> logs) {
this.logs = logs;
}
@Override
public void write(DataOutput out) throws IOException {
out.writeInt(logs.size());
for (RemoteEditLogWritable log : logs) {
log.write(out);
}
}
@Override
public void readFields(DataInput in) throws IOException {
int numLogs = in.readInt();
logs = Lists.newArrayList();
for (int i = 0; i < numLogs; i++) {
RemoteEditLogWritable log = new RemoteEditLogWritable();
log.readFields(in);
logs.add(log);
}
}
public static RemoteEditLogManifestWritable convert(
RemoteEditLogManifest editLogManifest) {
List<RemoteEditLogWritable> list = Lists.newArrayList();
for (RemoteEditLog log : editLogManifest.getLogs()) {
list.add(RemoteEditLogWritable.convert(log));
}
return new RemoteEditLogManifestWritable(list);
}
public RemoteEditLogManifest convert() {
List<RemoteEditLog> list = Lists.newArrayList();
for (RemoteEditLogWritable log : logs) {
list.add(log.convert());
}
return new RemoteEditLogManifest(list);
}
}

View File

@ -1,69 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
public class RemoteEditLogWritable implements Writable {
private long startTxId;
private long endTxId;
static { // register a ctor
WritableFactories.setFactory(RemoteEditLogWritable.class,
new WritableFactory() {
public Writable newInstance() {
return new RemoteEditLogWritable();
}
});
}
public RemoteEditLogWritable() {
}
public RemoteEditLogWritable(long startTxId, long endTxId) {
this.startTxId = startTxId;
this.endTxId = endTxId;
}
@Override
public void write(DataOutput out) throws IOException {
out.writeLong(startTxId);
out.writeLong(endTxId);
}
@Override
public void readFields(DataInput in) throws IOException {
startTxId = in.readLong();
endTxId = in.readLong();
}
public static RemoteEditLogWritable convert(RemoteEditLog log) {
return new RemoteEditLogWritable(log.getStartTxId(), log.getEndTxId());
}
public RemoteEditLog convert() {
return new RemoteEditLog(startTxId, endTxId);
}
}

View File

@ -1,86 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
import org.apache.hadoop.io.WritableUtils;
/**
* Common writable class for storage information.
*/
@InterfaceAudience.Private
public class StorageInfoWritable implements Writable {
private int layoutVersion;
private int namespaceID;
private String clusterID;
private long cTime;
public StorageInfoWritable () {
this(0, 0, "", 0L);
}
public StorageInfoWritable(int layoutV, int nsID, String cid, long cT) {
layoutVersion = layoutV;
clusterID = cid;
namespaceID = nsID;
cTime = cT;
}
/////////////////////////////////////////////////
// Writable
/////////////////////////////////////////////////
static {
WritableFactories.setFactory(StorageInfoWritable.class,
new WritableFactory() {
public Writable newInstance() {
return new StorageInfoWritable();
}
});
}
public void write(DataOutput out) throws IOException {
out.writeInt(layoutVersion);
out.writeInt(namespaceID);
WritableUtils.writeString(out, clusterID);
out.writeLong(cTime);
}
public void readFields(DataInput in) throws IOException {
layoutVersion = in.readInt();
namespaceID = in.readInt();
clusterID = WritableUtils.readString(in);
cTime = in.readLong();
}
public StorageInfo convert() {
return new StorageInfo(layoutVersion, namespaceID, clusterID, cTime);
}
public static StorageInfoWritable convert(StorageInfo from) {
return new StorageInfoWritable(from.getLayoutVersion(),
from.getNamespaceID(), from.getClusterID(), from.getCTime());
}
}

View File

@ -1,208 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.commons.codec.binary.Base64;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
/**
* The client-side form of the token.
*/
@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
@InterfaceStability.Stable
public class TokenWritable implements Writable {
private byte[] identifier;
private byte[] password;
private Text kind;
private Text service;
/**
* Construct a token from the components.
* @param identifier the token identifier
* @param password the token's password
* @param kind the kind of token
* @param service the service for this token
*/
public TokenWritable(byte[] identifier, byte[] password, Text kind, Text service) {
this.identifier = identifier;
this.password = password;
this.kind = kind;
this.service = service;
}
/**
* Default constructor
*/
public TokenWritable() {
this(new byte[0], new byte[0], new Text(), new Text());
}
/**
* Get the token identifier
* @return the token identifier
*/
public byte[] getIdentifier() {
return identifier;
}
/**
* Get the token password/secret
* @return the token password/secret
*/
public byte[] getPassword() {
return password;
}
/**
* Get the token kind
* @return the kind of the token
*/
public Text getKind() {
return kind;
}
/**
* Get the service on which the token is supposed to be used
* @return the service name
*/
public Text getService() {
return service;
}
/**
* Set the service on which the token is supposed to be used
* @param newService the service name
*/
public void setService(Text newService) {
service = newService;
}
@Override
public void readFields(DataInput in) throws IOException {
int len = WritableUtils.readVInt(in);
if (identifier == null || identifier.length != len) {
identifier = new byte[len];
}
in.readFully(identifier);
len = WritableUtils.readVInt(in);
if (password == null || password.length != len) {
password = new byte[len];
}
in.readFully(password);
kind.readFields(in);
service.readFields(in);
}
@Override
public void write(DataOutput out) throws IOException {
WritableUtils.writeVInt(out, identifier.length);
out.write(identifier);
WritableUtils.writeVInt(out, password.length);
out.write(password);
kind.write(out);
service.write(out);
}
/**
* Generate a string with the url-quoted base64 encoded serialized form
* of the Writable.
* @param obj the object to serialize
* @return the encoded string
* @throws IOException
*/
private static String encodeWritable(Writable obj) throws IOException {
DataOutputBuffer buf = new DataOutputBuffer();
obj.write(buf);
Base64 encoder = new Base64(0, null, true);
byte[] raw = new byte[buf.getLength()];
System.arraycopy(buf.getData(), 0, raw, 0, buf.getLength());
return encoder.encodeToString(raw);
}
/**
* Modify the writable to the value from the newValue
* @param obj the object to read into
* @param newValue the string with the url-safe base64 encoded bytes
* @throws IOException
*/
private static void decodeWritable(Writable obj,
String newValue) throws IOException {
Base64 decoder = new Base64(0, null, true);
DataInputBuffer buf = new DataInputBuffer();
byte[] decoded = decoder.decode(newValue);
buf.reset(decoded, decoded.length);
obj.readFields(buf);
}
/**
* Encode this token as a url safe string
* @return the encoded string
* @throws IOException
*/
public String encodeToUrlString() throws IOException {
return encodeWritable(this);
}
/**
* Decode the given url safe string into this token.
* @param newValue the encoded string
* @throws IOException
*/
public void decodeFromUrlString(String newValue) throws IOException {
decodeWritable(this, newValue);
}
private static void addBinaryBuffer(StringBuilder buffer, byte[] bytes) {
for (int idx = 0; idx < bytes.length; idx++) {
// if not the first, put a blank separator in
if (idx != 0) {
buffer.append(' ');
}
String num = Integer.toHexString(0xff & bytes[idx]);
// if it is only one digit, add a leading 0.
if (num.length() < 2) {
buffer.append('0');
}
buffer.append(num);
}
}
@Override
public String toString() {
StringBuilder buffer = new StringBuilder();
buffer.append("Ident: ");
addBinaryBuffer(buffer, identifier);
buffer.append(", Kind: ");
buffer.append(kind.toString());
buffer.append(", Service: ");
buffer.append(service.toString());
return buffer.toString();
}
}

View File

@ -1,140 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
/**
* Base upgrade upgradeStatus class.
*
* Describes status of current upgrade.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class UpgradeStatusReportWritable implements Writable {
protected int version;
protected short upgradeStatus;
protected boolean finalized;
public static UpgradeStatusReportWritable convert(
org.apache.hadoop.hdfs.server.common.UpgradeStatusReport r) {
if (r == null) return null;
return new UpgradeStatusReportWritable(
r.getVersion(), r.getUpgradeStatus(), r.isFinalized());
}
public static org.apache.hadoop.hdfs.server.common.UpgradeStatusReport
convert(UpgradeStatusReportWritable r) {
if (r == null) return null;
return new org.apache.hadoop.hdfs.server.common.UpgradeStatusReport(
r.getVersion(), r.getUpgradeStatus(), r.isFinalized());
}
public UpgradeStatusReportWritable() {
this(0, (short)0, false);
}
public UpgradeStatusReportWritable(int version, short status, boolean isFinalized) {
this.version = version;
this.upgradeStatus = status;
this.finalized = isFinalized;
}
/**
* Get the layout version of the currently running upgrade.
* @return layout version
*/
public int getVersion() {
return this.version;
}
/**
* Get upgrade upgradeStatus as a percentage of the total upgrade done.
*/
public short getUpgradeStatus() {
return upgradeStatus;
}
/**
* Is current upgrade finalized.
* @return true if finalized or false otherwise.
*/
public boolean isFinalized() {
return this.finalized;
}
/**
* Get upgradeStatus data as a text for reporting.
* Should be overloaded for a particular upgrade specific upgradeStatus data.
*
* @param details true if upgradeStatus details need to be included,
* false otherwise
* @return text
*/
public String getStatusText(boolean details) {
return "Upgrade for version " + getVersion()
+ (upgradeStatus<100 ?
" is in progress. Status = " + upgradeStatus + "%" :
" has been completed."
+ "\nUpgrade is " + (finalized ? "" : "not ")
+ "finalized.");
}
/**
* Print basic upgradeStatus details.
*/
@Override
public String toString() {
return getStatusText(false);
}
/////////////////////////////////////////////////
// Writable
/////////////////////////////////////////////////
static { // register a ctor
WritableFactories.setFactory
(UpgradeStatusReportWritable.class,
new WritableFactory() {
public Writable newInstance() { return new UpgradeStatusReportWritable(); }
});
}
// Note when upgrade has been finalized then the NN always
// returns a null as the report.
// hence the isFinalized is serialized (ugly)
@Override
public void write(DataOutput out) throws IOException {
out.writeInt(this.version);
out.writeShort(this.upgradeStatus);
}
@Override
public void readFields(DataInput in) throws IOException {
this.version = in.readInt();
this.upgradeStatus = in.readShort();
}
}

View File

@ -25,8 +25,6 @@
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol;
import org.apache.hadoop.hdfs.server.protocolR23Compatible.DatanodeWireProtocol;
import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.security.KerberosInfo;
@ -48,10 +46,11 @@ public interface DatanodeProtocol extends VersionedProtocol {
* to insulate from the protocol serialization.
*
* If you are adding/changing DN's interface then you need to
* change both this class and ALSO
* {@link DatanodeWireProtocol}.
* These changes need to be done in a compatible fashion as described in
* {@link ClientNamenodeWireProtocol}
* change both this class and ALSO related protocol buffer
* wire protocol definition in DatanodeProtocol.proto.
*
* For more details on protocol buffer wire protocol, please see
* .../org/apache/hadoop/hdfs/protocolPB/overview.html
*/
public static final long versionID = 28L;

View File

@ -25,9 +25,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
import org.apache.hadoop.hdfs.server.protocolR23Compatible.InterDatanodeWireProtocol;
import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.security.KerberosInfo;
@ -52,10 +50,11 @@ public interface InterDatanodeProtocol extends VersionedProtocol {
* serialization.
*
* If you are adding/changing DN's interface then you need to
* change both this class and ALSO
* {@link InterDatanodeWireProtocol}
* These changes need to be done in a compatible fashion as described in
* {@link ClientNamenodeWireProtocol}
* change both this class and ALSO related protocol buffer
* wire protocol definition in InterDatanodeProtocol.proto.
*
* For more details on protocol buffer wire protocol, please see
* .../org/apache/hadoop/hdfs/protocolPB/overview.html
*
* The log of historical changes can be retrieved from the svn).
* 6: Add block pool ID to Block

View File

@ -21,8 +21,6 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol;
import org.apache.hadoop.hdfs.protocolR23Compatible.JournalWireProtocol;
import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.security.KerberosInfo;
@ -41,10 +39,11 @@ public interface JournalProtocol extends VersionedProtocol {
* to insulate from the protocol serialization.
*
* If you are adding/changing DN's interface then you need to
* change both this class and ALSO
* {@link JournalWireProtocol}.
* These changes need to be done in a compatible fashion as described in
* {@link ClientNamenodeWireProtocol}
* change both this class and ALSO related protocol buffer
* wire protocol definition in JournalProtocol.proto.
*
* For more details on protocol buffer wire protocol, please see
* .../org/apache/hadoop/hdfs/protocolPB/overview.html
*/
public static final long versionID = 1L;

View File

@ -49,10 +49,11 @@ public interface NamenodeProtocol extends VersionedProtocol {
* NN server side to insulate from the protocol serialization.
*
* If you are adding/changing NN's interface then you need to
* change both this class and ALSO
* {@link org.apache.hadoop.hdfs.protocolR23Compatible.NamenodeWireProtocol}.
* These changes need to be done in a compatible fashion as described in
* {@link org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol}
* change both this class and ALSO related protocol buffer
* wire protocol definition in NamenodeProtocol.proto.
*
* For more details on protocol buffer wire protocol, please see
* .../org/apache/hadoop/hdfs/protocolPB/overview.html
*
* 6: Switch to txid-based file naming for image and edits
*/

View File

@ -1,107 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
/**
* Balancer bandwidth command instructs each datanode to change its value for
* the max amount of network bandwidth it may use during the block balancing
* operation.
*
* The Balancer Bandwidth Command contains the new bandwidth value as its
* payload. The bandwidth value is in bytes per second.
*/
public class BalancerBandwidthCommandWritable extends DatanodeCommandWritable {
private final static long BBC_DEFAULTBANDWIDTH = 0L;
private long bandwidth;
/**
* Balancer Bandwidth Command constructor. Sets bandwidth to 0.
*/
BalancerBandwidthCommandWritable() {
this(BBC_DEFAULTBANDWIDTH);
}
/**
* Balancer Bandwidth Command constructor.
* @param bandwidth Blanacer bandwidth in bytes per second.
*/
public BalancerBandwidthCommandWritable(long bandwidth) {
super(DatanodeWireProtocol.DNA_BALANCERBANDWIDTHUPDATE);
this.bandwidth = bandwidth;
}
/**
* Get current value of the max balancer bandwidth in bytes per second.
* @return bandwidth Blanacer bandwidth in bytes per second for this datanode.
*/
public long getBalancerBandwidthValue() {
return this.bandwidth;
}
// ///////////////////////////////////////////////
// Writable
// ///////////////////////////////////////////////
static { // register a ctor
WritableFactories.setFactory(BalancerBandwidthCommandWritable.class,
new WritableFactory() {
public Writable newInstance() {
return new BalancerBandwidthCommandWritable();
}
});
}
/**
* Writes the bandwidth payload to the Balancer Bandwidth Command packet.
* @param out DataOutput stream used for writing commands to the datanode.
* @throws IOException
*/
public void write(DataOutput out) throws IOException {
super.write(out);
out.writeLong(this.bandwidth);
}
/**
* Reads the bandwidth payload from the Balancer Bandwidth Command packet.
* @param in DataInput stream used for reading commands to the datanode.
* @throws IOException
*/
public void readFields(DataInput in) throws IOException {
super.readFields(in);
this.bandwidth = in.readLong();
}
@Override
public DatanodeCommand convert() {
return new BalancerBandwidthCommand(bandwidth);
}
public static DatanodeCommandWritable convert(BalancerBandwidthCommand cmd) {
return new BalancerBandwidthCommandWritable(cmd.getBalancerBandwidthValue());
}
}

View File

@ -1,142 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocolR23Compatible.BlockWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.DatanodeInfoWritable;
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
/****************************************************
* A BlockCommand is an instruction to a datanode regarding some blocks under
* its control. It tells the DataNode to either invalidate a set of indicated
* blocks, or to copy a set of indicated blocks to another DataNode.
*
****************************************************/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class BlockCommandWritable extends DatanodeCommandWritable {
/**
* This constant is used to indicate that the block deletion does not need
* explicit ACK from the datanode. When a block is put into the list of blocks
* to be deleted, it's size is set to this constant. We assume that no block
* would actually have this size. Otherwise, we would miss ACKs for blocks
* with such size. Positive number is used for compatibility reasons.
*/
public static final long NO_ACK = Long.MAX_VALUE;
String poolId;
BlockWritable blocks[];
DatanodeInfoWritable targets[][];
public BlockCommandWritable() {
}
/**
* Create BlockCommand for the given action
*
* @param blocks blocks related to the action
*/
public BlockCommandWritable(int action, String poolId, BlockWritable[] blocks,
DatanodeInfoWritable[][] targets) {
super(action);
this.poolId = poolId;
this.blocks = blocks;
this.targets = targets;
}
// /////////////////////////////////////////
// Writable
// /////////////////////////////////////////
static { // register a ctor
WritableFactories.setFactory(BlockCommandWritable.class,
new WritableFactory() {
public Writable newInstance() {
return new BlockCommandWritable();
}
});
}
@Override
public void write(DataOutput out) throws IOException {
super.write(out);
Text.writeString(out, poolId);
out.writeInt(blocks.length);
for (int i = 0; i < blocks.length; i++) {
blocks[i].write(out);
}
out.writeInt(targets.length);
for (int i = 0; i < targets.length; i++) {
out.writeInt(targets[i].length);
for (int j = 0; j < targets[i].length; j++) {
targets[i][j].write(out);
}
}
}
@Override
public void readFields(DataInput in) throws IOException {
super.readFields(in);
this.poolId = Text.readString(in);
this.blocks = new BlockWritable[in.readInt()];
for (int i = 0; i < blocks.length; i++) {
blocks[i] = new BlockWritable();
blocks[i].readFields(in);
}
this.targets = new DatanodeInfoWritable[in.readInt()][];
for (int i = 0; i < targets.length; i++) {
this.targets[i] = new DatanodeInfoWritable[in.readInt()];
for (int j = 0; j < targets[i].length; j++) {
targets[i][j] = new DatanodeInfoWritable();
targets[i][j].readFields(in);
}
}
}
@Override
public BlockCommand convert() {
DatanodeInfo[][] dinfo = new DatanodeInfo[targets.length][];
for (int i = 0; i < targets.length; i++) {
dinfo[i] = DatanodeInfoWritable.convertDatanodeInfo(targets[i]);
}
return new BlockCommand(getAction(), poolId, BlockWritable.convert(blocks),
dinfo);
}
public static BlockCommandWritable convert(BlockCommand cmd) {
if (cmd == null) return null;
DatanodeInfo[][] targets = cmd.getTargets();
DatanodeInfoWritable[][] dinfo = new DatanodeInfoWritable[targets.length][];
for (int i = 0; i < targets.length; i++) {
dinfo[i] = DatanodeInfoWritable.convertDatanodeInfo(targets[i]);
}
return new BlockCommandWritable(cmd.getAction(), cmd.getBlockPoolId(),
BlockWritable.convert(cmd.getBlocks()), dinfo);
}
}

View File

@ -1,118 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
/**
* BlockRecoveryCommand is an instruction to a data-node to recover the
* specified blocks.
*
* The data-node that receives this command treats itself as a primary data-node
* in the recover process.
*
* Block recovery is identified by a recoveryId, which is also the new
* generation stamp, which the block will have after the recovery succeeds.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class BlockRecoveryCommandWritable extends DatanodeCommandWritable {
Collection<RecoveringBlockWritable> recoveringBlocks;
/**
* Create empty BlockRecoveryCommand.
*/
public BlockRecoveryCommandWritable() { }
/**
* Create BlockRecoveryCommand with the specified capacity for recovering
* blocks.
*/
public BlockRecoveryCommandWritable(int capacity) {
this(new ArrayList<RecoveringBlockWritable>(capacity));
}
public BlockRecoveryCommandWritable(Collection<RecoveringBlockWritable> blocks) {
super(DatanodeWireProtocol.DNA_RECOVERBLOCK);
recoveringBlocks = blocks;
}
// /////////////////////////////////////////
// Writable
// /////////////////////////////////////////
static { // register a ctor
WritableFactories.setFactory(BlockRecoveryCommandWritable.class,
new WritableFactory() {
public Writable newInstance() {
return new BlockRecoveryCommandWritable();
}
});
}
public void write(DataOutput out) throws IOException {
super.write(out);
out.writeInt(recoveringBlocks.size());
for (RecoveringBlockWritable block : recoveringBlocks) {
block.write(out);
}
}
public void readFields(DataInput in) throws IOException {
super.readFields(in);
int numBlocks = in.readInt();
recoveringBlocks = new ArrayList<RecoveringBlockWritable>(numBlocks);
for (int i = 0; i < numBlocks; i++) {
RecoveringBlockWritable b = new RecoveringBlockWritable();
b.readFields(in);
recoveringBlocks.add(b);
}
}
@Override
public DatanodeCommand convert() {
Collection<RecoveringBlock> blks =
new ArrayList<RecoveringBlock>(recoveringBlocks.size());
for (RecoveringBlockWritable b : recoveringBlocks) {
blks.add(b.convert());
}
return new BlockRecoveryCommand(blks);
}
public static BlockRecoveryCommandWritable convert(BlockRecoveryCommand cmd) {
if (cmd == null) return null;
Collection<RecoveringBlockWritable> blks =
new ArrayList<RecoveringBlockWritable>(cmd.getRecoveringBlocks().size());
for (RecoveringBlock b : cmd.getRecoveringBlocks()) {
blks.add(RecoveringBlockWritable.convert(b));
}
return new BlockRecoveryCommandWritable(blks);
}
}

View File

@ -1,73 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
/**
* Class for translating DatanodeCommandWritable to and from DatanodeCommand.
*/
class DatanodeCommandHelper {
public static final Log LOG = LogFactory.getLog(DatanodeCommandHelper.class);
private DatanodeCommandHelper() {
/* Private constructor to prevent instantiation */
}
static DatanodeCommand convert(DatanodeCommandWritable cmd) {
return cmd.convert();
}
/**
* Given a subclass of {@link DatanodeCommand} return the corresponding
* writable type.
*/
static DatanodeCommandWritable convert(DatanodeCommand cmd) {
switch (cmd.getAction()) {
case DatanodeProtocol.DNA_BALANCERBANDWIDTHUPDATE:
return BalancerBandwidthCommandWritable
.convert((BalancerBandwidthCommand) cmd);
case DatanodeProtocol.DNA_FINALIZE:
return FinalizeCommandWritable.convert((FinalizeCommand)cmd);
case DatanodeProtocol.DNA_ACCESSKEYUPDATE:
return KeyUpdateCommandWritable.convert((KeyUpdateCommand)cmd);
case DatanodeProtocol.DNA_REGISTER:
return RegisterCommandWritable.REGISTER;
case DatanodeProtocol.DNA_TRANSFER:
case DatanodeProtocol.DNA_INVALIDATE:
return BlockCommandWritable.convert((BlockCommand)cmd);
case UpgradeCommand.UC_ACTION_START_UPGRADE:
return UpgradeCommandWritable.convert((UpgradeCommand)cmd);
case DatanodeProtocol.DNA_RECOVERBLOCK:
return BlockRecoveryCommandWritable.convert((BlockRecoveryCommand)cmd);
default:
LOG.warn("Unknown DatanodeCommand action - " + cmd.getAction());
return null;
}
}
}

View File

@ -1,58 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
/**
* Base class for data-node command.
* Issued by the name-node to notify data-nodes what should be done.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public abstract class DatanodeCommandWritable extends ServerCommandWritable {
public DatanodeCommandWritable() {
super();
}
DatanodeCommandWritable(int action) {
super(action);
}
/** Method to convert from writable type to internal type */
public abstract DatanodeCommand convert();
public static DatanodeCommandWritable[] convert(DatanodeCommand[] cmds) {
DatanodeCommandWritable[] ret = new DatanodeCommandWritable[cmds.length];
for (int i = 0; i < cmds.length; i++) {
ret[i] = DatanodeCommandHelper.convert(cmds[i]);
}
return ret;
}
public static DatanodeCommand[] convert(DatanodeCommandWritable[] cmds) {
if (cmds == null) return null;
DatanodeCommand[] ret = new DatanodeCommand[cmds.length];
for (int i = 0; i < cmds.length; i++) {
ret[i] = cmds[i].convert();
}
return ret;
}
}

View File

@ -1,170 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocolR23Compatible.DatanodeIDWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.ExtendedBlockWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.LocatedBlockWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.NamespaceInfoWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
/**
* This class is used on the server side. Calls come across the wire for the
* protocol family of Release 23 onwards. This class translates the R23 data
* types to the native data types used inside the NN as specified in the generic
* DatanodeProtocol.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class DatanodeProtocolServerSideTranslatorR23 implements
DatanodeWireProtocol {
final private DatanodeProtocol server;
/**
* Constructor
* @param server - the NN server
* @throws IOException
*/
public DatanodeProtocolServerSideTranslatorR23(DatanodeProtocol server)
throws IOException {
this.server = server;
}
/**
* The client side will redirect getProtocolSignature to
* getProtocolSignature2.
*
* However the RPC layer below on the Server side will call getProtocolVersion
* and possibly in the future getProtocolSignature. Hence we still implement
* it even though the end client's call will never reach here.
*/
@Override
public ProtocolSignature getProtocolSignature(String protocol,
long clientVersion, int clientMethodsHash) throws IOException {
/**
* Don't forward this to the server. The protocol version and signature is
* that of {@link DatanodeProtocol}
*
*/
if (!protocol.equals(RPC.getProtocolName(DatanodeWireProtocol.class))) {
throw new IOException("Namenode Serverside implements " +
RPC.getProtocolName(DatanodeWireProtocol.class) +
". The following requested protocol is unknown: " + protocol);
}
return ProtocolSignature.getProtocolSignature(clientMethodsHash,
DatanodeWireProtocol.versionID, DatanodeWireProtocol.class);
}
@Override
public ProtocolSignatureWritable
getProtocolSignature2(
String protocol, long clientVersion, int clientMethodsHash)
throws IOException {
/**
* Don't forward this to the server. The protocol version and signature is
* that of {@link DatanodeProtocol}
*/
return ProtocolSignatureWritable.convert(
this.getProtocolSignature(protocol, clientVersion, clientMethodsHash));
}
@Override
public long getProtocolVersion(String protocol, long clientVersion)
throws IOException {
if (protocol.equals(RPC.getProtocolName(DatanodeWireProtocol.class))) {
return DatanodeWireProtocol.versionID;
}
throw new IOException("Namenode Serverside implements " +
RPC.getProtocolName(DatanodeWireProtocol.class) +
". The following requested protocol is unknown: " + protocol);
}
@Override
public DatanodeRegistrationWritable registerDatanode(
DatanodeRegistrationWritable registration) throws IOException {
return DatanodeRegistrationWritable.convert(server
.registerDatanode(registration.convert()));
}
@Override
public DatanodeCommandWritable[] sendHeartbeat(
DatanodeRegistrationWritable registration, long capacity, long dfsUsed,
long remaining, long blockPoolUsed, int xmitsInProgress,
int xceiverCount, int failedVolumes) throws IOException {
return DatanodeCommandWritable.convert(server.sendHeartbeat(
registration.convert(), capacity, dfsUsed, remaining, blockPoolUsed,
xmitsInProgress, xceiverCount, failedVolumes));
}
@Override
public DatanodeCommandWritable blockReport(
DatanodeRegistrationWritable registration, String poolId, long[] blocks)
throws IOException {
return DatanodeCommandHelper.convert(server.blockReport(
registration.convert(), poolId, blocks));
}
@Override
public void blockReceivedAndDeleted(
DatanodeRegistrationWritable registration, String poolId,
ReceivedDeletedBlockInfoWritable[] receivedAndDeletedBlocks)
throws IOException {
server.blockReceivedAndDeleted(registration.convert(), poolId,
ReceivedDeletedBlockInfoWritable.convert(receivedAndDeletedBlocks));
}
@Override
public void errorReport(DatanodeRegistrationWritable registration,
int errorCode, String msg) throws IOException {
server.errorReport(registration.convert(), errorCode, msg);
}
@Override
public NamespaceInfoWritable versionRequest() throws IOException {
return NamespaceInfoWritable.convert(server.versionRequest());
}
@Override
public UpgradeCommandWritable processUpgradeCommand(
UpgradeCommandWritable comm) throws IOException {
return UpgradeCommandWritable.convert(server.processUpgradeCommand(comm.convert()));
}
@Override
public void reportBadBlocks(LocatedBlockWritable[] blocks) throws IOException {
server.reportBadBlocks(LocatedBlockWritable.convertLocatedBlock(blocks));
}
@Override
public void commitBlockSynchronization(ExtendedBlockWritable block,
long newgenerationstamp, long newlength, boolean closeFile,
boolean deleteblock, DatanodeIDWritable[] newtargets) throws IOException {
server.commitBlockSynchronization(
ExtendedBlockWritable.convertExtendedBlock(block), newgenerationstamp,
newlength, closeFile, deleteblock,
DatanodeIDWritable.convertDatanodeID(newtargets));
}
}

View File

@ -1,193 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import java.io.Closeable;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocolR23Compatible.DatanodeIDWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.ExtendedBlockWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.LocatedBlockWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
import org.apache.hadoop.io.retry.RetryPolicies;
import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.io.retry.RetryProxy;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
/**
* This class forwards NN's ClientProtocol calls as RPC calls to the NN server
* while translating from the parameter types used in ClientProtocol to those
* used in protocolR23Compatile.*.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class DatanodeProtocolTranslatorR23 implements
DatanodeProtocol, Closeable {
final private DatanodeWireProtocol rpcProxy;
private static DatanodeWireProtocol createNamenode(
InetSocketAddress nameNodeAddr, Configuration conf,
UserGroupInformation ugi) throws IOException {
return RPC.getProxy(DatanodeWireProtocol.class,
DatanodeWireProtocol.versionID, nameNodeAddr, ugi, conf,
NetUtils.getSocketFactory(conf, DatanodeWireProtocol.class));
}
/** Create a {@link NameNode} proxy */
static DatanodeWireProtocol createNamenodeWithRetry(
DatanodeWireProtocol rpcNamenode) {
RetryPolicy createPolicy = RetryPolicies
.retryUpToMaximumCountWithFixedSleep(5,
HdfsConstants.LEASE_SOFTLIMIT_PERIOD, TimeUnit.MILLISECONDS);
Map<Class<? extends Exception>, RetryPolicy> remoteExceptionToPolicyMap =
new HashMap<Class<? extends Exception>, RetryPolicy>();
remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class,
createPolicy);
Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
new HashMap<Class<? extends Exception>, RetryPolicy>();
exceptionToPolicyMap.put(RemoteException.class, RetryPolicies
.retryByRemoteException(RetryPolicies.TRY_ONCE_THEN_FAIL,
remoteExceptionToPolicyMap));
RetryPolicy methodPolicy = RetryPolicies.retryByException(
RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
Map<String, RetryPolicy> methodNameToPolicyMap = new HashMap<String, RetryPolicy>();
methodNameToPolicyMap.put("create", methodPolicy);
return (DatanodeWireProtocol) RetryProxy.create(
DatanodeWireProtocol.class, rpcNamenode, methodNameToPolicyMap);
}
public DatanodeProtocolTranslatorR23(InetSocketAddress nameNodeAddr,
Configuration conf) throws IOException {
UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
rpcProxy = createNamenodeWithRetry(createNamenode(nameNodeAddr, conf, ugi));
}
@Override
public void close() {
RPC.stopProxy(rpcProxy);
}
@Override
public long getProtocolVersion(String protocolName, long clientVersion)
throws IOException {
return rpcProxy.getProtocolVersion(protocolName, clientVersion);
}
@Override
public ProtocolSignature getProtocolSignature(String protocol,
long clientVersion, int clientMethodsHash) throws IOException {
return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
protocol, clientVersion, clientMethodsHash));
}
@Override
public DatanodeRegistration registerDatanode(DatanodeRegistration registration)
throws IOException {
return rpcProxy.registerDatanode(
DatanodeRegistrationWritable.convert(registration)).convert();
}
@Override
public DatanodeCommand[] sendHeartbeat(DatanodeRegistration registration,
long capacity, long dfsUsed, long remaining, long blockPoolUsed,
int xmitsInProgress, int xceiverCount, int failedVolumes)
throws IOException {
return DatanodeCommandWritable.convert(rpcProxy.sendHeartbeat(
DatanodeRegistrationWritable.convert(registration), capacity,
dfsUsed, remaining, blockPoolUsed, xmitsInProgress, xceiverCount,
failedVolumes));
}
@Override
public DatanodeCommand blockReport(DatanodeRegistration registration,
String poolId, long[] blocks) throws IOException {
return rpcProxy.blockReport(
DatanodeRegistrationWritable.convert(registration), poolId, blocks)
.convert();
}
@Override
public void blockReceivedAndDeleted(DatanodeRegistration registration,
String poolId, ReceivedDeletedBlockInfo[] receivedAndDeletedBlocks)
throws IOException {
rpcProxy.blockReceivedAndDeleted(
DatanodeRegistrationWritable.convert(registration), poolId,
ReceivedDeletedBlockInfoWritable.convert(receivedAndDeletedBlocks));
}
@Override
public void errorReport(DatanodeRegistration registration, int errorCode,
String msg) throws IOException {
rpcProxy.errorReport(DatanodeRegistrationWritable.convert(registration),
errorCode, msg);
}
@Override
public NamespaceInfo versionRequest() throws IOException {
return rpcProxy.versionRequest().convert();
}
@Override
public UpgradeCommand processUpgradeCommand(UpgradeCommand cmd)
throws IOException {
return rpcProxy.processUpgradeCommand(UpgradeCommandWritable.convert(cmd))
.convert();
}
@Override
public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
rpcProxy.reportBadBlocks(LocatedBlockWritable.convertLocatedBlock(blocks));
}
@Override
public void commitBlockSynchronization(ExtendedBlock block,
long newgenerationstamp, long newlength, boolean closeFile,
boolean deleteblock, DatanodeID[] newtargets) throws IOException {
rpcProxy.commitBlockSynchronization(
ExtendedBlockWritable.convertExtendedBlock(block), newgenerationstamp,
newlength, closeFile, deleteblock,
DatanodeIDWritable.convertDatanodeID(newtargets));
}
}

View File

@ -1,113 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocolR23Compatible.DatanodeIDWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.ExportedBlockKeysWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.StorageInfoWritable;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
/**
* DatanodeRegistration class contains all information the name-node needs
* to identify and verify a data-node when it contacts the name-node.
* This information is sent by data-node with each communication request.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class DatanodeRegistrationWritable implements Writable {
static { // register a ctor
WritableFactories.setFactory
(DatanodeRegistrationWritable.class,
new WritableFactory() {
public Writable newInstance() { return new DatanodeRegistrationWritable(); }
});
}
private DatanodeIDWritable datanodeId;
private StorageInfoWritable storageInfo;
private ExportedBlockKeysWritable exportedKeys;
/**
* Default constructor.
*/
public DatanodeRegistrationWritable() {
this("", new StorageInfo(), new ExportedBlockKeys());
}
/**
* Create DatanodeRegistration
*/
public DatanodeRegistrationWritable(String nodeName, StorageInfo info,
ExportedBlockKeys keys) {
this.datanodeId = new DatanodeIDWritable(nodeName);
this.storageInfo = StorageInfoWritable.convert(info);
this.exportedKeys = ExportedBlockKeysWritable.convert(keys);
}
/////////////////////////////////////////////////
// Writable
/////////////////////////////////////////////////
@Override
public void write(DataOutput out) throws IOException {
datanodeId.write(out);
//TODO: move it to DatanodeID once HADOOP-2797 has been committed
out.writeShort(datanodeId.ipcPort);
storageInfo.write(out);
exportedKeys.write(out);
}
@Override
public void readFields(DataInput in) throws IOException {
datanodeId.readFields(in);
//TODO: move it to DatanodeID once HADOOP-2797 has been committed
datanodeId.ipcPort = in.readShort() & 0x0000ffff;
storageInfo.readFields(in);
exportedKeys.readFields(in);
}
public DatanodeRegistration convert() {
DatanodeRegistration dnReg = new DatanodeRegistration(datanodeId.name,
storageInfo.convert(), exportedKeys.convert());
dnReg.setIpcPort(datanodeId.ipcPort);
return dnReg;
}
public static DatanodeRegistrationWritable convert(DatanodeRegistration dnReg) {
if (dnReg == null) return null;
DatanodeRegistrationWritable ret = new DatanodeRegistrationWritable(
dnReg.getName(), dnReg.storageInfo, dnReg.exportedKeys);
ret.datanodeId.ipcPort = dnReg.ipcPort;
return ret;
}
}

View File

@ -1,181 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol;
import org.apache.hadoop.hdfs.protocolR23Compatible.DatanodeIDWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.ExtendedBlockWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.LocatedBlockWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.NamespaceInfoWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.security.KerberosInfo;
/**********************************************************************
* Protocol that a DFS datanode uses to communicate with the NameNode.
* It's used to upload current load information and block reports.
*
* The only way a NameNode can communicate with a DataNode is by
* returning values from these functions.
*
**********************************************************************/
@KerberosInfo(
serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
clientPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
@InterfaceAudience.Private
public interface DatanodeWireProtocol extends VersionedProtocol {
/**
* The rules for changing this protocol are the same as that for
* {@link ClientNamenodeWireProtocol} - see that java file for details.
*/
public static final long versionID = 28L;
// error code
final static int NOTIFY = 0;
final static int DISK_ERROR = 1; // there are still valid volumes on DN
final static int INVALID_BLOCK = 2;
final static int FATAL_DISK_ERROR = 3; // no valid volumes left on DN
/**
* Determines actions that data node should perform
* when receiving a datanode command.
*/
final static int DNA_UNKNOWN = 0; // unknown action
final static int DNA_TRANSFER = 1; // transfer blocks to another datanode
final static int DNA_INVALIDATE = 2; // invalidate blocks
final static int DNA_SHUTDOWN = 3; // shutdown node
final static int DNA_REGISTER = 4; // re-register
final static int DNA_FINALIZE = 5; // finalize previous upgrade
final static int DNA_RECOVERBLOCK = 6; // request a block recovery
final static int DNA_ACCESSKEYUPDATE = 7; // update access key
final static int DNA_BALANCERBANDWIDTHUPDATE = 8; // update balancer bandwidth
/**
* Register Datanode.
* @return updated {@link DatanodeRegistrationWritable}, which contains
* new storageID if the datanode did not have one and
* registration ID for further communication.
*/
public DatanodeRegistrationWritable registerDatanode(
DatanodeRegistrationWritable registration) throws IOException;
/**
* sendHeartbeat() tells the NameNode that the DataNode is still
* alive and well. Includes some status info, too.
* It also gives the NameNode a chance to return
* an array of "DatanodeCommand" objects.
* A DatanodeCommand tells the DataNode to invalidate local block(s),
* or to copy them to other DataNodes, etc.
* @param registration datanode registration information
* @param capacity total storage capacity available at the datanode
* @param dfsUsed storage used by HDFS
* @param remaining remaining storage available for HDFS
* @param blockPoolUsed storage used by the block pool
* @param xmitsInProgress number of transfers from this datanode to others
* @param xceiverCount number of active transceiver threads
* @param failedVolumes number of failed volumes
* @throws IOException on error
*/
public DatanodeCommandWritable[] sendHeartbeat(
DatanodeRegistrationWritable registration, long capacity, long dfsUsed,
long remaining, long blockPoolUsed, int xmitsInProgress,
int xceiverCount, int failedVolumes) throws IOException;
/**
* blockReport() tells the NameNode about all the locally-stored blocks.
* The NameNode returns an array of Blocks that have become obsolete
* and should be deleted. This function is meant to upload *all*
* the locally-stored blocks. It's invoked upon startup and then
* infrequently afterwards.
* @param registration
* @param poolId - the block pool ID for the blocks
* @param blocks - the block list as an array of longs.
* Each block is represented as 2 longs.
* This is done instead of Block[] to reduce memory used by block reports.
*
* @return - the next command for DN to process.
* @throws IOException
*/
public DatanodeCommandWritable blockReport(
DatanodeRegistrationWritable registration, String poolId, long[] blocks)
throws IOException;
/**
* blockReceivedAndDeleted() allows the DataNode to tell the NameNode about
* recently-received and -deleted block data.
*
* For the case of received blocks, a hint for preferred replica to be
* deleted when there is any excessive blocks is provided.
* For example, whenever client code
* writes a new Block here, or another DataNode copies a Block to
* this DataNode, it will call blockReceived().
*/
public void blockReceivedAndDeleted(
DatanodeRegistrationWritable registration, String poolId,
ReceivedDeletedBlockInfoWritable[] receivedAndDeletedBlocks)
throws IOException;
/**
* errorReport() tells the NameNode about something that has gone
* awry. Useful for debugging.
*/
public void errorReport(DatanodeRegistrationWritable registration,
int errorCode, String msg) throws IOException;
public NamespaceInfoWritable versionRequest() throws IOException;
/**
* This is a very general way to send a command to the name-node during
* distributed upgrade process.
*
* The generosity is because the variety of upgrade commands is unpredictable.
* The reply from the name-node is also received in the form of an upgrade
* command.
*
* @return a reply in the form of an upgrade command
*/
UpgradeCommandWritable processUpgradeCommand(UpgradeCommandWritable comm)
throws IOException;
/**
* same as {@link ClientProtocol#reportBadBlocks(LocatedBlock[])}
* }
*/
public void reportBadBlocks(LocatedBlockWritable[] blocks) throws IOException;
/**
* Commit block synchronization in lease recovery
*/
public void commitBlockSynchronization(ExtendedBlockWritable block,
long newgenerationstamp, long newlength, boolean closeFile,
boolean deleteblock, DatanodeIDWritable[] newtargets) throws IOException;
/**
* This method is defined to get the protocol signature using
* the R23 protocol - hence we have added the suffix of 2 the method name
* to avoid conflict.
*/
public ProtocolSignatureWritable getProtocolSignature2(String protocol,
long clientVersion, int clientMethodsHash) throws IOException;
}

View File

@ -1,88 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
import org.apache.hadoop.io.WritableUtils;
/**
* A FinalizeCommand is an instruction from namenode to finalize the previous
* upgrade to a datanode
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class FinalizeCommandWritable extends DatanodeCommandWritable {
// /////////////////////////////////////////
// Writable
// /////////////////////////////////////////
static { // register a ctor
WritableFactories.setFactory(FinalizeCommandWritable.class,
new WritableFactory() {
public Writable newInstance() {
return new FinalizeCommandWritable();
}
});
}
String blockPoolId;
private FinalizeCommandWritable() {
this(null);
}
public FinalizeCommandWritable(String bpid) {
super(DatanodeWireProtocol.DNA_FINALIZE);
blockPoolId = bpid;
}
public String getBlockPoolId() {
return blockPoolId;
}
@Override
public void readFields(DataInput in) throws IOException {
blockPoolId = WritableUtils.readString(in);
}
@Override
public void write(DataOutput out) throws IOException {
WritableUtils.writeString(out, blockPoolId);
}
@Override
public DatanodeCommand convert() {
return new FinalizeCommand(blockPoolId);
}
public static FinalizeCommandWritable convert(FinalizeCommand cmd) {
if (cmd == null) {
return null;
}
return new FinalizeCommandWritable(cmd.getBlockPoolId());
}
}

View File

@ -1,116 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocolR23Compatible.ExtendedBlockWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
/**
* This class is used on the server side. Calls come across the wire for the
* protocol family of Release 23 onwards. This class translates the R23 data
* types to the internal data types used inside the DN as specified in the
* generic InterDatanodeProtocol.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class InterDatanodeProtocolServerSideTranslatorR23 implements
InterDatanodeWireProtocol {
final private InterDatanodeProtocol server;
/**
*
* @param server - datanode server
* @throws IOException
*/
public InterDatanodeProtocolServerSideTranslatorR23(
InterDatanodeProtocol server) throws IOException {
this.server = server;
}
/**
* the client side will redirect getProtocolSignature to
* getProtocolSignature2.
*
* However the RPC layer below on the Server side will call getProtocolVersion
* and possibly in the future getProtocolSignature. Hence we still implement
* it even though the end client's call will never reach here.
*/
@Override
public ProtocolSignature getProtocolSignature(String protocol,
long clientVersion, int clientMethodsHash) throws IOException {
/**
* Don't forward this to the server. The protocol version and signature is
* that of {@link InterDatanodeProtocol}
*/
if (!protocol.equals(RPC.getProtocolName(InterDatanodeWireProtocol.class))) {
throw new IOException("Datanode Serverside implements "
+ InterDatanodeWireProtocol.class
+ ". The following requested protocol is unknown: " + protocol);
}
return ProtocolSignature.getProtocolSignature(clientMethodsHash,
InterDatanodeWireProtocol.versionID, InterDatanodeWireProtocol.class);
}
@Override
public ProtocolSignatureWritable getProtocolSignature2(String protocol,
long clientVersion, int clientMethodsHash) throws IOException {
/**
* Don't forward this to the server. The protocol version and signature is
* that of {@link ClientNamenodeProtocol}
*/
return ProtocolSignatureWritable.convert(this.getProtocolSignature(
protocol, clientVersion, clientMethodsHash));
}
@Override
public long getProtocolVersion(String protocol, long clientVersion)
throws IOException {
if (protocol.equals(RPC.getProtocolName(InterDatanodeWireProtocol.class))) {
return InterDatanodeWireProtocol.versionID;
}
throw new IOException("Datanode Serverside implements "
+ InterDatanodeWireProtocol.class
+ ". The following requested protocol is unknown: " + protocol);
}
@Override
public ReplicaRecoveryInfoWritable initReplicaRecovery(
RecoveringBlockWritable rBlock) throws IOException {
return ReplicaRecoveryInfoWritable.convert(server
.initReplicaRecovery(rBlock.convert()));
}
@Override
public ExtendedBlockWritable updateReplicaUnderRecovery(
ExtendedBlockWritable oldBlock, long recoveryId, long newLength)
throws IOException {
ExtendedBlock b = ExtendedBlockWritable.convertExtendedBlock(oldBlock);
return ExtendedBlockWritable.convertExtendedBlock(server
.updateReplicaUnderRecovery(b, recoveryId, newLength));
}
}

View File

@ -1,96 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import java.io.IOException;
import java.net.InetSocketAddress;
import javax.net.SocketFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocolR23Compatible.ExtendedBlockWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.security.UserGroupInformation;
/**
* This class forwards InterDatanodeProtocol calls as RPC to the DN server while
* translating from the parameter types used in InterDatanodeProtocol to those
* used in protocolR23Compatile.*.
*/
@InterfaceAudience.Private
@InterfaceStability.Stable
public class InterDatanodeProtocolTranslatorR23 implements
InterDatanodeProtocol {
final private InterDatanodeWireProtocol rpcProxy;
/** used for testing */
public InterDatanodeProtocolTranslatorR23(InetSocketAddress addr,
UserGroupInformation ugi, Configuration conf, SocketFactory factory,
int socketTimeout)
throws IOException {
rpcProxy = createInterDatanodeProtocolProxy(addr, ugi, conf, factory,
socketTimeout);
}
static InterDatanodeWireProtocol createInterDatanodeProtocolProxy(
InetSocketAddress addr, UserGroupInformation ugi, Configuration conf,
SocketFactory factory, int socketTimeout) throws IOException {
return RPC.getProxy(InterDatanodeWireProtocol.class,
InterDatanodeWireProtocol.versionID, addr, ugi, conf, factory,
socketTimeout);
}
@Override
public ProtocolSignature getProtocolSignature(String protocolName,
long clientVersion, int clientMethodHash) throws IOException {
return ProtocolSignatureWritable.convert(rpcProxy.getProtocolSignature2(
protocolName, clientVersion, clientMethodHash));
}
@Override
public long getProtocolVersion(String protocolName, long clientVersion)
throws IOException {
return rpcProxy.getProtocolVersion(protocolName, clientVersion);
}
@Override
public ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock)
throws IOException {
return rpcProxy
.initReplicaRecovery(RecoveringBlockWritable.convert(rBlock)).convert();
}
@Override
public ExtendedBlock updateReplicaUnderRecovery(ExtendedBlock oldBlock,
long recoveryId, long newLength) throws IOException {
ExtendedBlockWritable eb = ExtendedBlockWritable
.convertExtendedBlock(oldBlock);
ExtendedBlockWritable b = rpcProxy.updateReplicaUnderRecovery(eb,
recoveryId, newLength);
return ExtendedBlockWritable.convertExtendedBlock(b);
}
}

View File

@ -1,73 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol;
import org.apache.hadoop.hdfs.protocolR23Compatible.ExtendedBlockWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.security.KerberosInfo;
/** An inter-datanode protocol for updating generation stamp
*/
@KerberosInfo(
serverPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY,
clientPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
@InterfaceAudience.Private
public interface InterDatanodeWireProtocol extends VersionedProtocol {
public static final Log LOG =
LogFactory.getLog(InterDatanodeWireProtocol.class);
/**
* The rules for changing this protocol are the same as that for
* {@link ClientNamenodeWireProtocol} - see that java file for details.
* 6: Add block pool ID to Block
*/
public static final long versionID = 6L;
/**
* Initialize a replica recovery.
*
* @return actual state of the replica on this data-node or
* null if data-node does not have the replica.
*/
ReplicaRecoveryInfoWritable initReplicaRecovery(RecoveringBlockWritable rBlock)
throws IOException;
/**
* Update replica with the new generation stamp and length.
*/
ExtendedBlockWritable updateReplicaUnderRecovery(
ExtendedBlockWritable oldBlock, long recoveryId, long newLength)
throws IOException;
/**
* This method is defined to get the protocol signature using
* the R23 protocol - hence we have added the suffix of 2 to the method name
* to avoid conflict.
*/
public ProtocolSignatureWritable getProtocolSignature2(
String protocol, long clientVersion, int clientMethodsHash)
throws IOException;
}

View File

@ -1,87 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocolR23Compatible.ExportedBlockKeysWritable;
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class KeyUpdateCommandWritable extends DatanodeCommandWritable {
private ExportedBlockKeysWritable keys;
KeyUpdateCommandWritable() {
this(new ExportedBlockKeysWritable());
}
public KeyUpdateCommandWritable(ExportedBlockKeysWritable keys) {
super(DatanodeWireProtocol.DNA_ACCESSKEYUPDATE);
this.keys = keys;
}
public ExportedBlockKeysWritable getExportedKeys() {
return this.keys;
}
// ///////////////////////////////////////////////
// Writable
// ///////////////////////////////////////////////
static { // register a ctor
WritableFactories.setFactory(KeyUpdateCommandWritable.class,
new WritableFactory() {
public Writable newInstance() {
return new KeyUpdateCommandWritable();
}
});
}
@Override
public void write(DataOutput out) throws IOException {
super.write(out);
keys.write(out);
}
@Override
public void readFields(DataInput in) throws IOException {
super.readFields(in);
keys.readFields(in);
}
@Override
public DatanodeCommand convert() {
return new KeyUpdateCommand(keys.convert());
}
public static KeyUpdateCommandWritable convert(KeyUpdateCommand cmd) {
if (cmd == null) {
return null;
}
return new KeyUpdateCommandWritable(ExportedBlockKeysWritable.convert(cmd
.getExportedKeys()));
}
}

View File

@ -1,95 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.hdfs.protocolR23Compatible.BlockWritable;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
/**
* A data structure to store Block and delHints together, used to send
* received/deleted ACKs.
*/
public class ReceivedDeletedBlockInfoWritable implements Writable {
BlockWritable block;
String delHints;
public final static String TODELETE_HINT = "-";
public ReceivedDeletedBlockInfoWritable() {
}
public ReceivedDeletedBlockInfoWritable(BlockWritable blk, String delHints) {
this.block = blk;
this.delHints = delHints;
}
@Override
public void write(DataOutput out) throws IOException {
this.block.write(out);
Text.writeString(out, this.delHints);
}
@Override
public void readFields(DataInput in) throws IOException {
this.block = new BlockWritable();
this.block.readFields(in);
this.delHints = Text.readString(in);
}
public String toString() {
return block.toString() + ", delHint: " + delHints;
}
public static ReceivedDeletedBlockInfo[] convert(
ReceivedDeletedBlockInfoWritable[] rdBlocks) {
ReceivedDeletedBlockInfo[] ret =
new ReceivedDeletedBlockInfo[rdBlocks.length];
for (int i = 0; i < rdBlocks.length; i++) {
ret[i] = rdBlocks[i].convert();
}
return ret;
}
public static ReceivedDeletedBlockInfoWritable[] convert(
ReceivedDeletedBlockInfo[] blocks) {
ReceivedDeletedBlockInfoWritable[] ret =
new ReceivedDeletedBlockInfoWritable[blocks.length];
for (int i = 0; i < blocks.length; i++) {
ret[i] = convert(blocks[i]);
}
return ret;
}
public ReceivedDeletedBlockInfo convert() {
return new ReceivedDeletedBlockInfo(block.convert(), delHints);
}
public static ReceivedDeletedBlockInfoWritable convert(
ReceivedDeletedBlockInfo b) {
if (b == null) return null;
return new ReceivedDeletedBlockInfoWritable(BlockWritable.convert(b
.getBlock()), b.getDelHints());
}
}

View File

@ -1,104 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocolR23Compatible.DatanodeInfoWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.ExtendedBlockWritable;
import org.apache.hadoop.hdfs.protocolR23Compatible.LocatedBlockWritable;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
/**
* This is a block with locations from which it should be recovered and the new
* generation stamp, which the block will have after successful recovery.
*
* The new generation stamp of the block, also plays role of the recovery id.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class RecoveringBlockWritable implements Writable {
private long newGenerationStamp;
private LocatedBlockWritable locatedBlock;
/**
* Create empty RecoveringBlock.
*/
public RecoveringBlockWritable() {
locatedBlock = new LocatedBlockWritable();
newGenerationStamp = -1L;
}
/**
* Create RecoveringBlock.
*/
public RecoveringBlockWritable(ExtendedBlockWritable b,
DatanodeInfoWritable[] locs, long newGS) {
locatedBlock = new LocatedBlockWritable(b, locs, -1, false);
this.newGenerationStamp = newGS;
}
// /////////////////////////////////////////
// Writable
// /////////////////////////////////////////
static { // register a ctor
WritableFactories.setFactory(RecoveringBlockWritable.class,
new WritableFactory() {
public Writable newInstance() {
return new RecoveringBlockWritable();
}
});
}
public void write(DataOutput out) throws IOException {
locatedBlock.write(out);
out.writeLong(newGenerationStamp);
}
public void readFields(DataInput in) throws IOException {
locatedBlock = new LocatedBlockWritable();
locatedBlock.readFields(in);
newGenerationStamp = in.readLong();
}
public RecoveringBlock convert() {
ExtendedBlockWritable eb = locatedBlock.getBlock();
DatanodeInfoWritable[] dnInfo = locatedBlock.getLocations();
return new RecoveringBlock(ExtendedBlockWritable.convertExtendedBlock(eb),
DatanodeInfoWritable.convertDatanodeInfo(dnInfo), newGenerationStamp);
}
public static RecoveringBlockWritable convert(RecoveringBlock rBlock) {
if (rBlock == null) {
return null;
}
ExtendedBlockWritable eb = ExtendedBlockWritable
.convertExtendedBlock(rBlock.getBlock());
DatanodeInfoWritable[] dnInfo = DatanodeInfoWritable
.convertDatanodeInfo(rBlock.getLocations());
return new RecoveringBlockWritable(eb, dnInfo,
rBlock.getNewGenerationStamp());
}
}

View File

@ -1,69 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
/**
* A RegisterCommand is an instruction to a datanode to register with the
* namenode.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class RegisterCommandWritable extends DatanodeCommandWritable {
public static final RegisterCommandWritable REGISTER =
new RegisterCommandWritable();
// /////////////////////////////////////////
// Writable
// /////////////////////////////////////////
static { // register a ctor
WritableFactories.setFactory(RegisterCommandWritable.class,
new WritableFactory() {
public Writable newInstance() {
return new RegisterCommandWritable();
}
});
}
public RegisterCommandWritable() {
super(DatanodeWireProtocol.DNA_REGISTER);
}
@Override
public void readFields(DataInput in) { /* Nothing to read */
}
@Override
public void write(DataOutput out) { /* Nothing to write */
}
@Override
public DatanodeCommand convert() {
return RegisterCommand.REGISTER;
}
}

View File

@ -1,87 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocolR23Compatible.BlockWritable;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
/**
* Replica recovery information.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class ReplicaRecoveryInfoWritable implements Writable {
private int originalState;
private BlockWritable block;
public ReplicaRecoveryInfoWritable() {
}
public ReplicaRecoveryInfoWritable(long blockId, long diskLen, long gs,
ReplicaState rState) {
block = new BlockWritable(blockId, diskLen, gs);
originalState = rState.getValue();
}
// /////////////////////////////////////////
// Writable
// /////////////////////////////////////////
static { // register a ctor
WritableFactories.setFactory(ReplicaRecoveryInfoWritable.class,
new WritableFactory() {
public Writable newInstance() {
return new ReplicaRecoveryInfoWritable();
}
});
}
@Override
public void readFields(DataInput in) throws IOException {
block = new BlockWritable();
block.readFields(in);
originalState = in.readInt();
}
@Override
public void write(DataOutput out) throws IOException {
block.write(out);
out.writeInt(originalState);
}
public static ReplicaRecoveryInfoWritable convert(ReplicaRecoveryInfo rrInfo) {
return new ReplicaRecoveryInfoWritable(rrInfo.getBlockId(),
rrInfo.getNumBytes(), rrInfo.getGenerationStamp(),
rrInfo.getOriginalReplicaState());
}
public ReplicaRecoveryInfo convert() {
return new ReplicaRecoveryInfo(block.getBlockId(), block.getNumBytes(),
block.getGenerationStamp(), ReplicaState.getState(originalState));
}
}

View File

@ -1,75 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.Writable;
/**
* Base class for a server command.
* Issued by the name-node to notify other servers what should be done.
* Commands are defined by actions defined in respective protocols.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public abstract class ServerCommandWritable implements Writable {
private int action;
/**
* Unknown server command constructor.
* Creates a command with action 0.
*/
public ServerCommandWritable() {
this(0);
}
/**
* Create a command for the specified action.
* Actions are protocol specific.
* @param action
*/
public ServerCommandWritable(int action) {
this.action = action;
}
/**
* Get server command action.
* @return action code.
*/
public int getAction() {
return this.action;
}
///////////////////////////////////////////
// Writable
///////////////////////////////////////////
@Override
public void write(DataOutput out) throws IOException {
out.writeInt(this.action);
}
@Override
public void readFields(DataInput in) throws IOException {
this.action = in.readInt();
}
}

View File

@ -1,106 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.protocolR23Compatible;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
/**
* This as a generic distributed upgrade command.
*
* During the upgrade cluster components send upgrade commands to each other
* in order to obtain or share information with them.
* It is supposed that each upgrade defines specific upgrade command by
* deriving them from this class.
* The upgrade command contains version of the upgrade, which is verified
* on the receiving side and current status of the upgrade.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class UpgradeCommandWritable extends DatanodeCommandWritable {
final static int UC_ACTION_UNKNOWN = DatanodeWireProtocol.DNA_UNKNOWN;
public final static int UC_ACTION_REPORT_STATUS = 100; // report upgrade status
public final static int UC_ACTION_START_UPGRADE = 101; // start upgrade
private int version;
private short upgradeStatus;
public UpgradeCommandWritable() {
super(UC_ACTION_UNKNOWN);
this.version = 0;
this.upgradeStatus = 0;
}
public UpgradeCommandWritable(int action, int version, short status) {
super(action);
this.version = version;
this.upgradeStatus = status;
}
public int getVersion() {
return this.version;
}
public short getCurrentStatus() {
return this.upgradeStatus;
}
/////////////////////////////////////////////////
// Writable
/////////////////////////////////////////////////
static { // register a ctor
WritableFactories.setFactory
(UpgradeCommandWritable.class,
new WritableFactory() {
public Writable newInstance() { return new UpgradeCommandWritable(); }
});
}
@Override
public void write(DataOutput out) throws IOException {
super.write(out);
out.writeInt(this.version);
out.writeShort(this.upgradeStatus);
}
@Override
public void readFields(DataInput in) throws IOException {
super.readFields(in);
this.version = in.readInt();
this.upgradeStatus = in.readShort();
}
@Override
public UpgradeCommand convert() {
return new UpgradeCommand(getAction(), version, upgradeStatus);
}
public static UpgradeCommandWritable convert(UpgradeCommand cmd) {
if (cmd == null) return null;
return new UpgradeCommandWritable(cmd.getAction(), cmd.getVersion(),
cmd.getCurrentStatus());
}
}

View File

@ -44,14 +44,9 @@
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
import org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
@ -62,7 +57,6 @@
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.hdfs.tools.DFSAdmin;
@ -70,11 +64,8 @@
import org.apache.hadoop.net.DNSToSwitchMapping;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.net.StaticMapping;
import org.apache.hadoop.security.RefreshUserMappingsProtocol;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.ProxyUsers;
import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
import org.apache.hadoop.tools.GetUserMappingsProtocol;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.ToolRunner;
@ -639,10 +630,6 @@ private void createFederatedNameNode(int nnIndex, Configuration conf,
nameNodes[nnIndex] = new NameNodeInfo(nn, new Configuration(conf));
}
private void setRpcEngine(Configuration conf, Class<?> protocol, Class<?> engine) {
conf.setClass("rpc.engine."+protocol.getName(), engine, Object.class);
}
/**
* @return URI of the namenode from a single namenode MiniDFSCluster
*/