Return-Path: X-Original-To: apmail-hadoop-hdfs-commits-archive@minotaur.apache.org Delivered-To: apmail-hadoop-hdfs-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 3EE509AA6 for ; Fri, 21 Oct 2011 19:03:45 +0000 (UTC) Received: (qmail 2216 invoked by uid 500); 21 Oct 2011 19:03:45 -0000 Delivered-To: apmail-hadoop-hdfs-commits-archive@hadoop.apache.org Received: (qmail 2174 invoked by uid 500); 21 Oct 2011 19:03:44 -0000 Mailing-List: contact hdfs-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hdfs-dev@hadoop.apache.org Delivered-To: mailing list hdfs-commits@hadoop.apache.org Received: (qmail 2166 invoked by uid 99); 21 Oct 2011 19:03:44 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 21 Oct 2011 19:03:44 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 21 Oct 2011 19:03:40 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 465AD23888EA; Fri, 21 Oct 2011 19:03:20 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1187505 - in /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ src/main/java/org/apache/hadoop/hdfs/security/token/block/ src/main/java/org/apache/hadoop/hdfs/server/nameno... Date: Fri, 21 Oct 2011 19:03:19 -0000 To: hdfs-commits@hadoop.apache.org From: suresh@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20111021190320.465AD23888EA@eris.apache.org> Author: suresh Date: Fri Oct 21 19:03:18 2011 New Revision: 1187505 URL: http://svn.apache.org/viewvc?rev=1187505&view=rev Log: HDFS-2480. Separate datatypes for NamenodeProtocol. Contributed by Suresh Srinivas. Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlockWritable.java hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlocksWithLocationsWritable.java hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/CheckpointSignatureWritable.java hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/ExportedBlockKeysWritable.java hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeCommandWritable.java hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeProtocolServerSideTranslatorR23.java hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeProtocolTranslatorR23.java hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeWireProtocol.java hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamespaceInfoWritable.java hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogManifestWritable.java hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogWritable.java Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/DatanodeInfoWritable.java hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/ExportedBlockKeys.java hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1187505&r1=1187504&r2=1187505&view=diff ============================================================================== --- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original) +++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Fri Oct 21 19:03:18 2011 @@ -61,7 +61,9 @@ Trunk (unreleased changes) HDFS-2395. Add a root element in the JSON responses of webhdfs. (szetszwo) - HDFS-2459. Separate datatypes for Journal Protocol. (suresh) + HDFS-2459. Separate datatypes for JournalProtocol. (suresh) + + HDFS-2480. Separate datatypes for NamenodeProtocol. (suresh) HDFS-2181 Separate HDFS Client wire protocol data types (sanjay) Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlockWritable.java URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlockWritable.java?rev=1187505&view=auto ============================================================================== --- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlockWritable.java (added) +++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlockWritable.java Fri Oct 21 19:03:18 2011 @@ -0,0 +1,82 @@ +/** + * 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); + } +} Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlocksWithLocationsWritable.java URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlocksWithLocationsWritable.java?rev=1187505&view=auto ============================================================================== --- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlocksWithLocationsWritable.java (added) +++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/BlocksWithLocationsWritable.java Fri Oct 21 19:03:18 2011 @@ -0,0 +1,129 @@ +/** + * 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, RetryPolicy> remoteExceptionToPolicyMap = + new HashMap, RetryPolicy>(); + remoteExceptionToPolicyMap.put(AlreadyBeingCreatedException.class, + createPolicy); + + Map, RetryPolicy> exceptionToPolicyMap = + new HashMap, RetryPolicy>(); + exceptionToPolicyMap.put(RemoteException.class, RetryPolicies + .retryByRemoteException(RetryPolicies.TRY_ONCE_THEN_FAIL, + remoteExceptionToPolicyMap)); + RetryPolicy methodPolicy = RetryPolicies.retryByException( + RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap); + Map methodNameToPolicyMap = new HashMap(); + + methodNameToPolicyMap.put("create", methodPolicy); + + return (NamenodeWireProtocol) RetryProxy.create( + NamenodeWireProtocol.class, rpcNamenode, methodNameToPolicyMap); + } + + public NamenodeProtocolTranslatorR23(InetSocketAddress nameNodeAddr, + Configuration conf, UserGroupInformation ugi) throws IOException { + rpcProxyWithoutRetry = createNamenode(nameNodeAddr, conf, ugi); + rpcProxy = createNamenodeWithRetry(rpcProxyWithoutRetry); + } + + public Object getProxyWithoutRetry() { + return rpcProxyWithoutRetry; + } + + public void close() { + RPC.stopProxy(rpcProxyWithoutRetry); + } + + @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(); + } +} Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeWireProtocol.java URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeWireProtocol.java?rev=1187505&view=auto ============================================================================== --- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeWireProtocol.java (added) +++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamenodeWireProtocol.java Fri Oct 21 19:03:18 2011 @@ -0,0 +1,169 @@ +/** + * 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 datanode + * whose total size equals size. + * + * @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; +} + Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamespaceInfoWritable.java URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamespaceInfoWritable.java?rev=1187505&view=auto ============================================================================== --- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamespaceInfoWritable.java (added) +++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/NamespaceInfoWritable.java Fri Oct 21 19:03:18 2011 @@ -0,0 +1,100 @@ +/** + * 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); + } +} Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogManifestWritable.java URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogManifestWritable.java?rev=1187505&view=auto ============================================================================== --- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogManifestWritable.java (added) +++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogManifestWritable.java Fri Oct 21 19:03:18 2011 @@ -0,0 +1,90 @@ +/** + * 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 logs; + + static { // register a ctor + WritableFactories.setFactory(RemoteEditLogManifestWritable.class, + new WritableFactory() { + public Writable newInstance() { + return new RemoteEditLogManifestWritable(); + } + }); + } + + public RemoteEditLogManifestWritable() { + } + + public RemoteEditLogManifestWritable(List 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 list = Lists.newArrayList(); + for (RemoteEditLog log : editLogManifest.getLogs()) { + list.add(RemoteEditLogWritable.convert(log)); + } + return new RemoteEditLogManifestWritable(list); + } + + public RemoteEditLogManifest convert() { + List list = Lists.newArrayList(); + for (RemoteEditLogWritable log : logs) { + list.add(log.convert()); + } + return new RemoteEditLogManifest(list); + } +} Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogWritable.java URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogWritable.java?rev=1187505&view=auto ============================================================================== --- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogWritable.java (added) +++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolR23Compatible/RemoteEditLogWritable.java Fri Oct 21 19:03:18 2011 @@ -0,0 +1,69 @@ +/** + * 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); + } +} Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/ExportedBlockKeys.java URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/ExportedBlockKeys.java?rev=1187505&r1=1187504&r2=1187505&view=diff ============================================================================== --- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/ExportedBlockKeys.java (original) +++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/ExportedBlockKeys.java Fri Oct 21 19:03:18 2011 @@ -43,7 +43,7 @@ public class ExportedBlockKeys implement this(false, 0, 0, new BlockKey(), new BlockKey[0]); } - ExportedBlockKeys(boolean isBlockTokenEnabled, long keyUpdateInterval, + public ExportedBlockKeys(boolean isBlockTokenEnabled, long keyUpdateInterval, long tokenLifetime, BlockKey currentKey, BlockKey[] allKeys) { this.isBlockTokenEnabled = isBlockTokenEnabled; this.keyUpdateInterval = keyUpdateInterval; Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java?rev=1187505&r1=1187504&r2=1187505&view=diff ============================================================================== --- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java (original) +++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java Fri Oct 21 19:03:18 2011 @@ -37,9 +37,7 @@ public class CheckpointSignature extends implements WritableComparable { private static final String FIELD_SEPARATOR = ":"; private static final int NUM_FIELDS = 7; - String blockpoolID = ""; - long mostRecentCheckpointTxId; long curSegmentTxId; @@ -67,6 +65,14 @@ public class CheckpointSignature extends blockpoolID = fields[i++]; } + public CheckpointSignature(StorageInfo info, String blockpoolID, + long mostRecentCheckpointTxId, long curSegmentTxId) { + super(info); + this.blockpoolID = blockpoolID; + this.mostRecentCheckpointTxId = mostRecentCheckpointTxId; + this.curSegmentTxId = curSegmentTxId; + } + /** * Get the cluster id from CheckpointSignature * @return the cluster id @@ -83,6 +89,14 @@ public class CheckpointSignature extends return blockpoolID; } + public long getMostRecentCheckpointTxId() { + return mostRecentCheckpointTxId; + } + + public long getCurSegmentTxId() { + return curSegmentTxId; + } + /** * Set the block pool id of CheckpointSignature. * Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java?rev=1187505&r1=1187504&r2=1187505&view=diff ============================================================================== --- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java (original) +++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java Fri Oct 21 19:03:18 2011 @@ -38,9 +38,21 @@ import org.apache.hadoop.security.Kerber @InterfaceAudience.Private public interface NamenodeProtocol extends VersionedProtocol { /** - * Compared to the previous version the following changes have been introduced: - * (Only the latest change is reflected. - * The log of historical changes can be retrieved from the svn). + * Until version 6L, this class served as both + * the client interface to the NN AND the RPC protocol used to + * communicate with the NN. + * + * Post version 70 (release 23 of Hadoop), the protocol is implemented in + * {@literal ../protocolR23Compatible/ClientNamenodeWireProtocol} + * + * 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.NamenodeWireProtocol}. + * These changes need to be done in a compatible fashion as described in + * {@link org.apache.hadoop.hdfs.protocolR23Compatible.ClientNamenodeWireProtocol} * * 6: Switch to txid-based file naming for image and edits */ @@ -62,7 +74,7 @@ public interface NamenodeProtocol extend * @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 + * @throws IOException if size is less than or equal to 0 or datanode does not exist */ public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)