svn merge -c 1241007 from trunk for HDFS-2895.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23-PB@1241605 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
9de9d30160
commit
e4b02079af
|
@ -85,6 +85,9 @@ Release 0.23-PB - Unreleased
|
|||
HDFS-2801. Provide a method in client side translators to check for a
|
||||
methods supported in underlying protocol. (jitendra)
|
||||
|
||||
HDFS-2895. Remove Writable wire protocol types and translators to
|
||||
complete transition to protocol buffers. (suresh)
|
||||
|
||||
BUG FIXES
|
||||
|
||||
HDFS-2481 Unknown protocol: org.apache.hadoop.hdfs.protocol.ClientProtocol (sanjay)
|
||||
|
|
|
@ -51,10 +51,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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -127,8 +127,6 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetSaf
|
|||
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;
|
||||
|
|
|
@ -58,7 +58,6 @@ 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.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 @@ public class DatanodeProtocolClientSideTranslatorPB implements
|
|||
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 */
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -1,145 +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);
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
|
@ -1,101 +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;
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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)
|
||||
+ "}";
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -25,8 +25,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|||
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;
|
||||
|
||||
|
|
|
@ -25,9 +25,7 @@ import org.apache.commons.logging.LogFactory;
|
|||
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
|
||||
|
|
|
@ -21,8 +21,6 @@ import java.io.IOException;
|
|||
|
||||
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;
|
||||
|
||||
|
|
|
@ -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
|
||||
*/
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -1,147 +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.List;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
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.blockmanagement.DatanodeDescriptor.BlockTargetPair;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
|
||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
|
||||
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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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()));
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -44,14 +44,9 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.FileUtil;
|
||||
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.datanode.SimulatedFSDataset;
|
|||
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.metrics2.lib.DefaultMetricsSystem;
|
|||
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 @@ public class MiniDFSCluster {
|
|||
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
|
||||
*/
|
||||
|
|
Loading…
Reference in New Issue