Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RefreshUserMappingsProtocolServerSideTranslatorPB.java Wed Feb 29 23:47:49 2012
@@ -24,9 +24,6 @@ import org.apache.hadoop.hdfs.protocol.p
import org.apache.hadoop.hdfs.protocol.proto.RefreshUserMappingsProtocolProtos.RefreshSuperUserGroupsConfigurationResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.RefreshUserMappingsProtocolProtos.RefreshUserToGroupsMappingsRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.RefreshUserMappingsProtocolProtos.RefreshUserToGroupsMappingsResponseProto;
-import org.apache.hadoop.hdfs.protocolR23Compatible.ProtocolSignatureWritable;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.security.RefreshUserMappingsProtocol;
import com.google.protobuf.RpcController;
@@ -66,40 +63,4 @@ public class RefreshUserMappingsProtocol
return RefreshSuperUserGroupsConfigurationResponseProto.newBuilder()
.build();
}
-
- @Override
- public long getProtocolVersion(String protocol, long clientVersion)
- throws IOException {
- return RPC.getProtocolVersion(RefreshUserMappingsProtocolPB.class);
- }
-
- @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 RefreshUserMappingsProtocol}
- */
- if (!protocol.equals(RPC
- .getProtocolName(RefreshUserMappingsProtocolPB.class))) {
- throw new IOException("Namenode Serverside implements "
- + RPC.getProtocolName(RefreshUserMappingsProtocolPB.class)
- + ". The following requested protocol is unknown: " + protocol);
- }
-
- return ProtocolSignature.getProtocolSignature(clientMethodsHash,
- RPC.getProtocolVersion(RefreshUserMappingsProtocolPB.class),
- RefreshUserMappingsProtocolPB.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 RefreshUserMappingsProtocolPB}
- */
- return ProtocolSignatureWritable.convert(this.getProtocolSignature(
- protocol, clientVersion, clientMethodsHash));
- }
}
Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java Wed Feb 29 23:47:49 2012
@@ -626,9 +626,12 @@ public class DataNode extends Configured
// DatanodeProtocol namenode,
SecureResources resources
) throws IOException {
- if(UserGroupInformation.isSecurityEnabled() && resources == null)
- throw new RuntimeException("Cannot start secure cluster without " +
- "privileged resources.");
+ if(UserGroupInformation.isSecurityEnabled() && resources == null) {
+ if (!conf.getBoolean("ignore.secure.ports.for.testing", false)) {
+ throw new RuntimeException("Cannot start secure cluster without "
+ + "privileged resources.");
+ }
+ }
// settings global for all BPs in the Data Node
this.secureResources = resources;
@@ -1780,25 +1783,6 @@ public class DataNode extends Configured
return new ExtendedBlock(oldBlock.getBlockPoolId(), r);
}
- @Override
- public long getProtocolVersion(String protocol, long clientVersion
- ) throws IOException {
- if (protocol.equals(InterDatanodeProtocol.class.getName())) {
- return InterDatanodeProtocol.versionID;
- } else if (protocol.equals(ClientDatanodeProtocol.class.getName())) {
- return ClientDatanodeProtocol.versionID;
- }
- throw new IOException("Unknown protocol to " + getClass().getSimpleName()
- + ": " + protocol);
- }
-
- @Override
- public ProtocolSignature getProtocolSignature(String protocol,
- long clientVersion, int clientMethodsHash) throws IOException {
- return ProtocolSignature.getProtocolSignature(
- this, protocol, clientVersion, clientMethodsHash);
- }
-
/** A convenient class used in block recovery */
static class BlockRecord {
final DatanodeID id;
Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java Wed Feb 29 23:47:49 2012
@@ -41,8 +41,8 @@ import org.apache.hadoop.hdfs.server.pro
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.ipc.RPC;
import org.apache.hadoop.ipc.StandbyException;
+import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
@@ -229,15 +229,6 @@ public class BackupNode extends NameNode
nnRpcAddress = nn.nnRpcAddress;
}
- @Override
- public long getProtocolVersion(String protocol, long clientVersion)
- throws IOException {
- if (protocol.equals(JournalProtocol.class.getName())) {
- return JournalProtocol.versionID;
- }
- return super.getProtocolVersion(protocol, clientVersion);
- }
-
/////////////////////////////////////////////////////
// BackupNodeProtocol implementation for backup node.
/////////////////////////////////////////////////////
Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditsDoubleBuffer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditsDoubleBuffer.java?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditsDoubleBuffer.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditsDoubleBuffer.java Wed Feb 29 23:47:49 2012
@@ -86,7 +86,7 @@ class EditsDoubleBuffer {
}
boolean shouldForceSync() {
- return bufReady.size() >= initBufferSize;
+ return bufCurrent.size() >= initBufferSize;
}
DataOutputBuffer getCurrentBuf() {
Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java Wed Feb 29 23:47:49 2012
@@ -249,8 +249,6 @@ public class FSDirectory implements Clos
+" to the file system");
return null;
}
- // add create file record to log, record new generation stamp
- fsImage.getEditLog().logOpenFile(path, newNode);
if(NameNode.stateChangeLog.isDebugEnabled()) {
NameNode.stateChangeLog.debug("DIR* FSDirectory.addFile: "
Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java Wed Feb 29 23:47:49 2012
@@ -821,6 +821,14 @@ public class FSEditLog {
this.runtime = runtime;
this.journalSet.setRuntimeForTesting(runtime);
}
+
+ /**
+ * Used only by tests.
+ */
+ @VisibleForTesting
+ void setMetricsForTests(NameNodeMetrics metrics) {
+ this.metrics = metrics;
+ }
/**
* Return a manifest of what finalized edit logs are available
Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java Wed Feb 29 23:47:49 2012
@@ -112,9 +112,8 @@ public class FSEditLogLoader {
long recentOpcodeOffsets[] = new long[4];
Arrays.fill(recentOpcodeOffsets, -1);
+ long txId = expectedStartingTxId - 1;
try {
- long txId = expectedStartingTxId - 1;
-
try {
while (true) {
FSEditLogOp op;
@@ -123,7 +122,8 @@ public class FSEditLogLoader {
break;
}
} catch (IOException ioe) {
- String errorMessage = formatEditLogReplayError(in, recentOpcodeOffsets);
+ long badTxId = txId + 1; // because txId hasn't been incremented yet
+ String errorMessage = formatEditLogReplayError(in, recentOpcodeOffsets, badTxId);
FSImage.LOG.error(errorMessage);
throw new EditLogInputException(errorMessage,
ioe, numEdits);
@@ -131,12 +131,12 @@ public class FSEditLogLoader {
recentOpcodeOffsets[(int)(numEdits % recentOpcodeOffsets.length)] =
in.getPosition();
if (LayoutVersion.supports(Feature.STORED_TXIDS, logVersion)) {
- long thisTxId = op.txid;
- if (thisTxId != txId + 1) {
+ long expectedTxId = txId + 1;
+ txId = op.txid;
+ if (txId != expectedTxId) {
throw new IOException("Expected transaction ID " +
- (txId + 1) + " but got " + thisTxId);
+ expectedTxId + " but got " + txId);
}
- txId = thisTxId;
}
incrOpCount(op.opCode, opCounts);
@@ -145,7 +145,7 @@ public class FSEditLogLoader {
} catch (Throwable t) {
// Catch Throwable because in the case of a truly corrupt edits log, any
// sort of error might be thrown (NumberFormat, NullPointer, EOF, etc.)
- String errorMessage = formatEditLogReplayError(in, recentOpcodeOffsets);
+ String errorMessage = formatEditLogReplayError(in, recentOpcodeOffsets, txId);
FSImage.LOG.error(errorMessage);
throw new IOException(errorMessage, t);
}
@@ -265,12 +265,22 @@ public class FSEditLogLoader {
updateBlocks(fsDir, addCloseOp, oldFile);
// Now close the file
- INodeFileUnderConstruction ucFile = (INodeFileUnderConstruction) oldFile;
+ if (!oldFile.isUnderConstruction() &&
+ logVersion <= LayoutVersion.BUGFIX_HDFS_2991_VERSION) {
+ // There was a bug (HDFS-2991) in hadoop < 0.23.1 where OP_CLOSE
+ // could show up twice in a row. But after that version, this
+ // should be fixed, so we should treat it as an error.
+ throw new IOException(
+ "File is not under construction: " + addCloseOp.path);
+ }
// One might expect that you could use removeLease(holder, path) here,
// but OP_CLOSE doesn't serialize the holder. So, remove by path.
- fsNamesys.leaseManager.removeLeaseWithPrefixPath(addCloseOp.path);
- INodeFile newFile = ucFile.convertToInodeFile();
- fsDir.replaceNode(addCloseOp.path, ucFile, newFile);
+ if (oldFile.isUnderConstruction()) {
+ INodeFileUnderConstruction ucFile = (INodeFileUnderConstruction) oldFile;
+ fsNamesys.leaseManager.removeLeaseWithPrefixPath(addCloseOp.path);
+ INodeFile newFile = ucFile.convertToInodeFile();
+ fsDir.replaceNode(addCloseOp.path, ucFile, newFile);
+ }
break;
}
case OP_SET_REPLICATION: {
@@ -431,9 +441,10 @@ public class FSEditLogLoader {
}
private static String formatEditLogReplayError(EditLogInputStream in,
- long recentOpcodeOffsets[]) {
+ long recentOpcodeOffsets[], long txid) {
StringBuilder sb = new StringBuilder();
sb.append("Error replaying edit log at offset " + in.getPosition());
+ sb.append(" on transaction ID ").append(txid);
if (recentOpcodeOffsets[0] != -1) {
Arrays.sort(recentOpcodeOffsets);
sb.append("\nRecent opcode offsets:");
Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java Wed Feb 29 23:47:49 2012
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.na
import java.util.zip.CheckedInputStream;
import java.util.zip.Checksum;
+import java.util.Arrays;
import java.util.EnumMap;
import org.apache.hadoop.fs.ChecksumException;
@@ -305,6 +306,36 @@ public abstract class FSEditLogOp {
}
return blocks;
}
+
+ public String stringifyMembers() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("[length=");
+ builder.append(length);
+ builder.append(", path=");
+ builder.append(path);
+ builder.append(", replication=");
+ builder.append(replication);
+ builder.append(", mtime=");
+ builder.append(mtime);
+ builder.append(", atime=");
+ builder.append(atime);
+ builder.append(", blockSize=");
+ builder.append(blockSize);
+ builder.append(", blocks=");
+ builder.append(Arrays.toString(blocks));
+ builder.append(", permissions=");
+ builder.append(permissions);
+ builder.append(", clientName=");
+ builder.append(clientName);
+ builder.append(", clientMachine=");
+ builder.append(clientMachine);
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class AddOp extends AddCloseOp {
@@ -315,6 +346,14 @@ public abstract class FSEditLogOp {
static AddOp getInstance() {
return (AddOp)opInstances.get().get(OP_ADD);
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("AddOp ");
+ builder.append(stringifyMembers());
+ return builder.toString();
+ }
}
static class CloseOp extends AddCloseOp {
@@ -325,6 +364,14 @@ public abstract class FSEditLogOp {
static CloseOp getInstance() {
return (CloseOp)opInstances.get().get(OP_CLOSE);
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("CloseOp ");
+ builder.append(stringifyMembers());
+ return builder.toString();
+ }
}
static class SetReplicationOp extends FSEditLogOp {
@@ -366,6 +413,21 @@ public abstract class FSEditLogOp {
this.replication = readShort(in);
}
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("SetReplicationOp [path=");
+ builder.append(path);
+ builder.append(", replication=");
+ builder.append(replication);
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class ConcatDeleteOp extends FSEditLogOp {
@@ -440,6 +502,25 @@ public abstract class FSEditLogOp {
this.timestamp = readLong(in);
}
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("ConcatDeleteOp [length=");
+ builder.append(length);
+ builder.append(", trg=");
+ builder.append(trg);
+ builder.append(", srcs=");
+ builder.append(Arrays.toString(srcs));
+ builder.append(", timestamp=");
+ builder.append(timestamp);
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class RenameOldOp extends FSEditLogOp {
@@ -497,6 +578,25 @@ public abstract class FSEditLogOp {
this.timestamp = readLong(in);
}
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("RenameOldOp [length=");
+ builder.append(length);
+ builder.append(", src=");
+ builder.append(src);
+ builder.append(", dst=");
+ builder.append(dst);
+ builder.append(", timestamp=");
+ builder.append(timestamp);
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class DeleteOp extends FSEditLogOp {
@@ -545,6 +645,23 @@ public abstract class FSEditLogOp {
this.timestamp = readLong(in);
}
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("DeleteOp [length=");
+ builder.append(length);
+ builder.append(", path=");
+ builder.append(path);
+ builder.append(", timestamp=");
+ builder.append(timestamp);
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class MkdirOp extends FSEditLogOp {
@@ -623,6 +740,25 @@ public abstract class FSEditLogOp {
this.permissions = null;
}
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("MkdirOp [length=");
+ builder.append(length);
+ builder.append(", path=");
+ builder.append(path);
+ builder.append(", timestamp=");
+ builder.append(timestamp);
+ builder.append(", permissions=");
+ builder.append(permissions);
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class SetGenstampOp extends FSEditLogOp {
@@ -652,6 +788,19 @@ public abstract class FSEditLogOp {
throws IOException {
this.genStamp = FSImageSerialization.readLong(in);
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("SetGenstampOp [genStamp=");
+ builder.append(genStamp);
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
@SuppressWarnings("deprecation")
@@ -676,6 +825,17 @@ public abstract class FSEditLogOp {
//Datanodes are not persistent any more.
FSImageSerialization.DatanodeImage.skipOne(in);
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("DatanodeAddOp [opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
@SuppressWarnings("deprecation")
@@ -701,6 +861,17 @@ public abstract class FSEditLogOp {
nodeID.readFields(in);
//Datanodes are not persistent any more.
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("DatanodeRemoveOp [opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class SetPermissionsOp extends FSEditLogOp {
@@ -738,6 +909,21 @@ public abstract class FSEditLogOp {
this.src = FSImageSerialization.readString(in);
this.permissions = FsPermission.read(in);
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("SetPermissionsOp [src=");
+ builder.append(src);
+ builder.append(", permissions=");
+ builder.append(permissions);
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class SetOwnerOp extends FSEditLogOp {
@@ -783,6 +969,23 @@ public abstract class FSEditLogOp {
this.username = FSImageSerialization.readString_EmptyAsNull(in);
this.groupname = FSImageSerialization.readString_EmptyAsNull(in);
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("SetOwnerOp [src=");
+ builder.append(src);
+ builder.append(", username=");
+ builder.append(username);
+ builder.append(", groupname=");
+ builder.append(groupname);
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class SetNSQuotaOp extends FSEditLogOp {
@@ -809,6 +1012,21 @@ public abstract class FSEditLogOp {
this.src = FSImageSerialization.readString(in);
this.nsQuota = FSImageSerialization.readLong(in);
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("SetNSQuotaOp [src=");
+ builder.append(src);
+ builder.append(", nsQuota=");
+ builder.append(nsQuota);
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class ClearNSQuotaOp extends FSEditLogOp {
@@ -833,6 +1051,19 @@ public abstract class FSEditLogOp {
throws IOException {
this.src = FSImageSerialization.readString(in);
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("ClearNSQuotaOp [src=");
+ builder.append(src);
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class SetQuotaOp extends FSEditLogOp {
@@ -878,6 +1109,23 @@ public abstract class FSEditLogOp {
this.nsQuota = FSImageSerialization.readLong(in);
this.dsQuota = FSImageSerialization.readLong(in);
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("SetQuotaOp [src=");
+ builder.append(src);
+ builder.append(", nsQuota=");
+ builder.append(nsQuota);
+ builder.append(", dsQuota=");
+ builder.append(dsQuota);
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class TimesOp extends FSEditLogOp {
@@ -936,6 +1184,25 @@ public abstract class FSEditLogOp {
this.atime = readLong(in);
}
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("TimesOp [length=");
+ builder.append(length);
+ builder.append(", path=");
+ builder.append(path);
+ builder.append(", mtime=");
+ builder.append(mtime);
+ builder.append(", atime=");
+ builder.append(atime);
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class SymlinkOp extends FSEditLogOp {
@@ -1011,6 +1278,29 @@ public abstract class FSEditLogOp {
}
this.permissionStatus = PermissionStatus.read(in);
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("SymlinkOp [length=");
+ builder.append(length);
+ builder.append(", path=");
+ builder.append(path);
+ builder.append(", value=");
+ builder.append(value);
+ builder.append(", mtime=");
+ builder.append(mtime);
+ builder.append(", atime=");
+ builder.append(atime);
+ builder.append(", permissionStatus=");
+ builder.append(permissionStatus);
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class RenameOp extends FSEditLogOp {
@@ -1097,6 +1387,27 @@ public abstract class FSEditLogOp {
}
return new BytesWritable(bytes);
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("RenameOp [length=");
+ builder.append(length);
+ builder.append(", src=");
+ builder.append(src);
+ builder.append(", dst=");
+ builder.append(dst);
+ builder.append(", timestamp=");
+ builder.append(timestamp);
+ builder.append(", options=");
+ builder.append(Arrays.toString(options));
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class ReassignLeaseOp extends FSEditLogOp {
@@ -1142,6 +1453,23 @@ public abstract class FSEditLogOp {
this.path = FSImageSerialization.readString(in);
this.newHolder = FSImageSerialization.readString(in);
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("ReassignLeaseOp [leaseHolder=");
+ builder.append(leaseHolder);
+ builder.append(", path=");
+ builder.append(path);
+ builder.append(", newHolder=");
+ builder.append(newHolder);
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class GetDelegationTokenOp extends FSEditLogOp {
@@ -1185,6 +1513,21 @@ public abstract class FSEditLogOp {
this.expiryTime = readLong(in);
}
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("GetDelegationTokenOp [token=");
+ builder.append(token);
+ builder.append(", expiryTime=");
+ builder.append(expiryTime);
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class RenewDelegationTokenOp extends FSEditLogOp {
@@ -1228,6 +1571,21 @@ public abstract class FSEditLogOp {
this.expiryTime = readLong(in);
}
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("RenewDelegationTokenOp [token=");
+ builder.append(token);
+ builder.append(", expiryTime=");
+ builder.append(expiryTime);
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class CancelDelegationTokenOp extends FSEditLogOp {
@@ -1259,6 +1617,19 @@ public abstract class FSEditLogOp {
this.token = new DelegationTokenIdentifier();
this.token.readFields(in);
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("CancelDelegationTokenOp [token=");
+ builder.append(token);
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class UpdateMasterKeyOp extends FSEditLogOp {
@@ -1289,6 +1660,19 @@ public abstract class FSEditLogOp {
this.key = new DelegationKey();
this.key.readFields(in);
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("UpdateMasterKeyOp [key=");
+ builder.append(key);
+ builder.append(", opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class LogSegmentOp extends FSEditLogOp {
@@ -1311,6 +1695,17 @@ public abstract class FSEditLogOp {
void writeFields(DataOutputStream out) throws IOException {
// no data stored
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("LogSegmentOp [opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static class InvalidOp extends FSEditLogOp {
@@ -1331,6 +1726,17 @@ public abstract class FSEditLogOp {
throws IOException {
// nothing to read
}
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("InvalidOp [opCode=");
+ builder.append(opCode);
+ builder.append(", txid=");
+ builder.append(txid);
+ builder.append("]");
+ return builder.toString();
+ }
}
static private short readShort(DataInputStream in) throws IOException {
Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Wed Feb 29 23:47:49 2012
@@ -1639,6 +1639,9 @@ public class FSNamesystem implements Nam
"Unable to add file to namespace.");
}
leaseManager.addLease(newNode.getClientName(), src);
+
+ // record file record in log, record new generation stamp
+ getEditLog().logOpenFile(src, newNode);
if (NameNode.stateChangeLog.isDebugEnabled()) {
NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: "
+"add "+src+" to namespace for "+holder);
@@ -1684,11 +1687,11 @@ public class FSNamesystem implements Nam
dir.replaceNode(src, node, cons);
leaseManager.addLease(cons.getClientName(), src);
+ LocatedBlock ret = blockManager.convertLastBlockToUnderConstruction(cons);
if (writeToEditLog) {
getEditLog().logOpenFile(src, cons);
}
-
- return blockManager.convertLastBlockToUnderConstruction(cons);
+ return ret;
}
/**
Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java Wed Feb 29 23:47:49 2012
@@ -17,6 +17,10 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HANDLER_COUNT_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HANDLER_COUNT_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_HANDLER_COUNT_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_HANDLER_COUNT_KEY;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.MAX_PATH_DEPTH;
import static org.apache.hadoop.hdfs.protocol.HdfsConstants.MAX_PATH_LENGTH;
@@ -37,8 +41,6 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
-
import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.ha.HealthCheckFailedException;
import org.apache.hadoop.ha.ServiceFailedException;
@@ -50,44 +52,43 @@ import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HDFSPolicyProvider;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
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.HdfsConstants;
+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.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
-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.proto.ClientNamenodeProtocolProtos.ClientNamenodeProtocol;
-import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.NamenodeProtocolService;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeProtocolService;
import org.apache.hadoop.hdfs.protocol.proto.GetUserMappingsProtocolProtos.GetUserMappingsProtocolService;
+import org.apache.hadoop.hdfs.protocol.proto.NamenodeProtocolProtos.NamenodeProtocolService;
import org.apache.hadoop.hdfs.protocol.proto.RefreshAuthorizationPolicyProtocolProtos.RefreshAuthorizationPolicyProtocolService;
import org.apache.hadoop.hdfs.protocol.proto.RefreshUserMappingsProtocolProtos.RefreshUserMappingsProtocolService;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
+import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolServerSideTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.GetUserMappingsProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.GetUserMappingsProtocolServerSideTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB;
-import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB;
-import org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.RefreshAuthorizationPolicyProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.RefreshAuthorizationPolicyProtocolServerSideTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.RefreshUserMappingsProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.RefreshUserMappingsProtocolServerSideTranslatorPB;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
@@ -99,7 +100,6 @@ import org.apache.hadoop.hdfs.server.pro
import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand;
import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
@@ -112,7 +112,6 @@ import org.apache.hadoop.hdfs.server.pro
import org.apache.hadoop.io.EnumSetWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
-import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
import org.apache.hadoop.ipc.Server;
@@ -120,14 +119,11 @@ import org.apache.hadoop.ipc.WritableRpc
import org.apache.hadoop.net.Node;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.Groups;
-import org.apache.hadoop.security.RefreshUserMappingsProtocol;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AuthorizationException;
import org.apache.hadoop.security.authorize.ProxyUsers;
-import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
-import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
-import org.apache.hadoop.tools.GetUserMappingsProtocol;
+import org.apache.hadoop.security.token.Token;
import com.google.protobuf.BlockingService;
@@ -299,36 +295,6 @@ class NameNodeRpcServer implements Namen
InetSocketAddress getRpcAddress() {
return clientRpcAddress;
}
-
- @Override // VersionedProtocol
- public ProtocolSignature getProtocolSignature(String protocol,
- long clientVersion, int clientMethodsHash) throws IOException {
- return ProtocolSignature.getProtocolSignature(
- this, protocol, clientVersion, clientMethodsHash);
- }
-
- @Override
- public long getProtocolVersion(String protocol,
- long clientVersion) throws IOException {
- if (protocol.equals(ClientProtocol.class.getName())) {
- throw new IOException("Old Namenode Client protocol is not supported:" +
- protocol + "Switch your clientside to " + ClientNamenodeProtocol.class);
- } else if (protocol.equals(DatanodeProtocol.class.getName())){
- return DatanodeProtocol.versionID;
- } else if (protocol.equals(NamenodeProtocol.class.getName())){
- return NamenodeProtocol.versionID;
- } else if (protocol.equals(RefreshAuthorizationPolicyProtocol.class.getName())){
- return RefreshAuthorizationPolicyProtocol.versionID;
- } else if (protocol.equals(RefreshUserMappingsProtocol.class.getName())){
- return RefreshUserMappingsProtocol.versionID;
- } else if (protocol.equals(GetUserMappingsProtocol.class.getName())){
- return GetUserMappingsProtocol.versionID;
- } else if (protocol.equals(HAServiceProtocol.class.getName())) {
- return HAServiceProtocol.versionID;
- } else {
- throw new IOException("Unknown protocol to name node: " + protocol);
- }
- }
/////////////////////////////////////////////////////
// NamenodeProtocol
Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java Wed Feb 29 23:47:49 2012
@@ -25,7 +25,6 @@ import org.apache.hadoop.hdfs.DFSConfigK
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.ipc.VersionedProtocol;
import org.apache.hadoop.security.KerberosInfo;
/**********************************************************************
@@ -40,7 +39,7 @@ import org.apache.hadoop.security.Kerber
serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
clientPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
@InterfaceAudience.Private
-public interface DatanodeProtocol extends VersionedProtocol {
+public interface DatanodeProtocol {
/**
* This class is used by both the Namenode (client) and BackupNode (server)
* to insulate from the protocol serialization.
Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java Wed Feb 29 23:47:49 2012
@@ -35,7 +35,7 @@ import org.apache.hadoop.security.Kerber
serverPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY,
clientPrincipal = DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY)
@InterfaceAudience.Private
-public interface InterDatanodeProtocol extends VersionedProtocol {
+public interface InterDatanodeProtocol {
public static final Log LOG = LogFactory.getLog(InterDatanodeProtocol.class);
/**
Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/JournalProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/JournalProtocol.java?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/JournalProtocol.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/JournalProtocol.java Wed Feb 29 23:47:49 2012
@@ -32,7 +32,7 @@ import org.apache.hadoop.security.Kerber
serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
@InterfaceAudience.Private
-public interface JournalProtocol extends VersionedProtocol {
+public interface JournalProtocol {
/**
*
* This class is used by both the Namenode (client) and BackupNode (server)
Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java Wed Feb 29 23:47:49 2012
@@ -36,7 +36,7 @@ import org.apache.hadoop.security.Kerber
serverPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY,
clientPrincipal = DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY)
@InterfaceAudience.Private
-public interface NamenodeProtocol extends VersionedProtocol {
+public interface NamenodeProtocol {
/**
* Until version 6L, this class served as both
* the client interface to the NN AND the RPC protocol used to
Propchange: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/native/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed Feb 29 23:47:49 2012
@@ -1,4 +1,4 @@
-/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native:1152502-1294758
+/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native:1152502-1295332
/hadoop/core/branches/branch-0.19/hdfs/src/main/native:713112
/hadoop/core/branches/branch-0.19/mapred/src/c++/libhdfs:713112
/hadoop/core/trunk/src/c++/libhdfs:776175-784663
Propchange: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed Feb 29 23:47:49 2012
@@ -1,4 +1,4 @@
-/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode:1159757-1294758
+/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode:1159757-1295332
/hadoop/core/branches/branch-0.19/hdfs/src/main/webapps/datanode:713112
/hadoop/core/branches/branch-0.19/hdfs/src/webapps/datanode:713112
/hadoop/core/trunk/src/webapps/datanode:776175-784663
Propchange: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed Feb 29 23:47:49 2012
@@ -1,4 +1,4 @@
-/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs:1152502-1294758
+/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs:1152502-1295332
/hadoop/core/branches/branch-0.19/hdfs/src/main/webapps/hdfs:713112
/hadoop/core/branches/branch-0.19/hdfs/src/webapps/hdfs:713112
/hadoop/core/trunk/src/webapps/hdfs:776175-784663
Propchange: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed Feb 29 23:47:49 2012
@@ -1,4 +1,4 @@
-/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary:1152502-1294758
+/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary:1152502-1295332
/hadoop/core/branches/branch-0.19/hdfs/src/main/webapps/secondary:713112
/hadoop/core/branches/branch-0.19/hdfs/src/webapps/secondary:713112
/hadoop/core/trunk/src/webapps/secondary:776175-784663
Propchange: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/hdfs/
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Wed Feb 29 23:47:49 2012
@@ -1,4 +1,4 @@
-/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/hdfs:1159757-1294758
+/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/hdfs:1159757-1295332
/hadoop/core/branches/branch-0.19/hdfs/src/test/hdfs:713112
/hadoop/core/trunk/src/test/hdfs:776175-785643
/hadoop/hdfs/branches/HDFS-1052/src/test/hdfs:987665-1095512
Copied: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java (from r1295332, hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java)
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java?p2=hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java&p1=hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java&r1=1295332&r2=1295342&rev=1295342&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppendRestart.java Wed Feb 29 23:47:49 2012
@@ -90,7 +90,7 @@ public class TestFileAppendRestart {
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
FileSystem fs = cluster.getFileSystem();
File editLog =
- new File(FSImageTestUtil.getNameNodeCurrentDirs(cluster).get(0),
+ new File(FSImageTestUtil.getNameNodeCurrentDirs(cluster, 0).get(0),
NNStorage.getInProgressEditsFileName(1));
EnumMap<FSEditLogOpCodes, Holder<Integer>> counts;
@@ -98,17 +98,27 @@ public class TestFileAppendRestart {
writeAndAppend(fs, p1, BLOCK_SIZE, BLOCK_SIZE);
counts = FSImageTestUtil.countEditLogOpTypes(editLog);
- assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_ADD).held);
+ // OP_ADD to create file
+ // OP_ADD for first block
+ // OP_CLOSE to close file
+ // OP_ADD to reopen file
+ // OP_ADD for second block
+ // OP_CLOSE to close file
+ assertEquals(4, (int)counts.get(FSEditLogOpCodes.OP_ADD).held);
assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_CLOSE).held);
Path p2 = new Path("/not-block-boundaries");
writeAndAppend(fs, p2, BLOCK_SIZE/2, BLOCK_SIZE);
counts = FSImageTestUtil.countEditLogOpTypes(editLog);
- // We get *3* OP_ADDS from this test rather than two. The first
- // OP_ADD comes from re-opening the file to establish the lease,
- // the second comes from the updatePipeline call when the block
- // itself has its generation stamp incremented
- assertEquals(5, (int)counts.get(FSEditLogOpCodes.OP_ADD).held);
+ // OP_ADD to create file
+ // OP_ADD for first block
+ // OP_CLOSE to close file
+ // OP_ADD to re-establish the lease
+ // OP_ADD from the updatePipeline call (increments genstamp of last block)
+ // OP_ADD at the start of the second block
+ // OP_CLOSE to close file
+ // Total: 5 OP_ADDs and 2 OP_CLOSEs in addition to the ones above
+ assertEquals(9, (int)counts.get(FSEditLogOpCodes.OP_ADD).held);
assertEquals(4, (int)counts.get(FSEditLogOpCodes.OP_CLOSE).held);
cluster.restartNameNode();
Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestClientProtocolWithDelegationToken.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestClientProtocolWithDelegationToken.java?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestClientProtocolWithDelegationToken.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestClientProtocolWithDelegationToken.java Wed Feb 29 23:47:49 2012
@@ -18,39 +18,31 @@
package org.apache.hadoop.hdfs.security;
-import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.anyString;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.when;
import java.net.InetSocketAddress;
import java.security.PrivilegedExceptionAction;
-import org.apache.commons.logging.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
-
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-
-import org.apache.hadoop.ipc.Client;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.ipc.Client;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SaslInputStream;
import org.apache.hadoop.security.SaslRpcClient;
import org.apache.hadoop.security.SaslRpcServer;
import org.apache.hadoop.security.UserGroupInformation;
-
+import org.apache.hadoop.security.token.Token;
import org.apache.log4j.Level;
import org.junit.Test;
@@ -80,12 +72,6 @@ public class TestClientProtocolWithDeleg
public void testDelegationTokenRpc() throws Exception {
ClientProtocol mockNN = mock(ClientProtocol.class);
FSNamesystem mockNameSys = mock(FSNamesystem.class);
- when(mockNN.getProtocolVersion(anyString(), anyLong())).thenReturn(
- ClientProtocol.versionID);
- doReturn(ProtocolSignature.getProtocolSignature(
- mockNN, ClientProtocol.class.getName(),
- ClientProtocol.versionID, 0))
- .when(mockNN).getProtocolSignature(anyString(), anyLong(), anyInt());
DelegationTokenSecretManager sm = new DelegationTokenSecretManager(
DFSConfigKeys.DFS_NAMENODE_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT,
Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java Wed Feb 29 23:47:49 2012
@@ -23,13 +23,8 @@ import static org.junit.Assert.assertEqu
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyInt;
-import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
@@ -51,12 +46,12 @@ import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ClientDatanodeProtocolService;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.GetReplicaVisibleLengthResponseProto;
@@ -65,7 +60,6 @@ import org.apache.hadoop.hdfs.protocolPB
import org.apache.hadoop.io.TestWritable;
import org.apache.hadoop.ipc.Client;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
-import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.net.NetUtils;
@@ -222,13 +216,6 @@ public class TestBlockToken {
private Server createMockDatanode(BlockTokenSecretManager sm,
Token<BlockTokenIdentifier> token) throws IOException, ServiceException {
ClientDatanodeProtocolPB mockDN = mock(ClientDatanodeProtocolPB.class);
- when(mockDN.getProtocolVersion(anyString(), anyLong())).thenReturn(
- RPC.getProtocolVersion(ClientDatanodeProtocolPB.class));
- doReturn(
- ProtocolSignature.getProtocolSignature(mockDN,
- ClientDatanodeProtocolPB.class.getName(),
- RPC.getProtocolVersion(ClientDatanodeProtocolPB.class), 0)).when(
- mockDN).getProtocolSignature(anyString(), anyLong(), anyInt());
BlockTokenIdentifier id = sm.createIdentifier();
id.readFields(new DataInputStream(new ByteArrayInputStream(token
Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java Wed Feb 29 23:47:49 2012
@@ -26,6 +26,7 @@ import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
+import java.util.EnumMap;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -44,6 +45,7 @@ import org.apache.hadoop.hdfs.server.com
import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
import org.apache.hadoop.hdfs.server.namenode.FSImageStorageInspector.FSImageFile;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
+import org.apache.hadoop.hdfs.util.Holder;
import org.apache.hadoop.hdfs.util.MD5FileUtils;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
@@ -195,6 +197,7 @@ public abstract class FSImageTestUtil {
return editLog;
}
+
/**
* Create an aborted in-progress log in the given directory, containing
* only a specified number of "mkdirs" operations.
@@ -217,6 +220,35 @@ public abstract class FSImageTestUtil {
}
/**
+ * @param editLog a path of an edit log file
+ * @return the count of each type of operation in the log file
+ * @throws Exception if there is an error reading it
+ */
+ public static EnumMap<FSEditLogOpCodes,Holder<Integer>> countEditLogOpTypes(
+ File editLog) throws Exception {
+ EnumMap<FSEditLogOpCodes, Holder<Integer>> opCounts =
+ new EnumMap<FSEditLogOpCodes, Holder<Integer>>(FSEditLogOpCodes.class);
+
+ EditLogInputStream elis = new EditLogFileInputStream(editLog);
+ try {
+ FSEditLogOp op;
+ while ((op = elis.readOp()) != null) {
+ Holder<Integer> i = opCounts.get(op.opCode);
+ if (i == null) {
+ i = new Holder<Integer>(0);
+ opCounts.put(op.opCode, i);
+ }
+ i.held++;
+ }
+ } finally {
+ IOUtils.closeStream(elis);
+ }
+
+ return opCounts;
+ }
+
+
+ /**
* Assert that all of the given directories have the same newest filename
* for fsimage that they hold the same data.
*/
Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java Wed Feb 29 23:47:49 2012
@@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.server.nam
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.hdfs.server.namenode.NNStorage;
+import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.log4j.Level;
@@ -818,6 +819,40 @@ public class TestEditLog extends TestCas
log.close();
}
}
+
+ /**
+ * Regression test for HDFS-1112/HDFS-3020. Ensures that, even if
+ * logSync isn't called periodically, the edit log will sync itself.
+ */
+ public void testAutoSync() throws Exception {
+ File logDir = new File(TEST_DIR, "testAutoSync");
+ logDir.mkdirs();
+ FSEditLog log = FSImageTestUtil.createStandaloneEditLog(logDir);
+
+ String oneKB = StringUtils.byteToHexString(
+ new byte[500]);
+
+ try {
+ log.openForWrite();
+ NameNodeMetrics mockMetrics = Mockito.mock(NameNodeMetrics.class);
+ log.setMetricsForTests(mockMetrics);
+
+ for (int i = 0; i < 400; i++) {
+ log.logDelete(oneKB, 1L);
+ }
+ // After ~400KB, we're still within the 512KB buffer size
+ Mockito.verify(mockMetrics, Mockito.times(0)).addSync(Mockito.anyLong());
+
+ // After ~400KB more, we should have done an automatic sync
+ for (int i = 0; i < 400; i++) {
+ log.logDelete(oneKB, 1L);
+ }
+ Mockito.verify(mockMetrics, Mockito.times(1)).addSync(Mockito.anyLong());
+
+ } finally {
+ log.close();
+ }
+ }
/**
* Tests the getEditLogManifest function using mock storage for a number
Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java Wed Feb 29 23:47:49 2012
@@ -90,15 +90,17 @@ public class TestFSEditLogLoader {
}
rwf.close();
- String expectedErrorMessage = "^Error replaying edit log at offset \\d+\n";
- expectedErrorMessage += "Recent opcode offsets: (\\d+\\s*){4}$";
+ StringBuilder bld = new StringBuilder();
+ bld.append("^Error replaying edit log at offset \\d+");
+ bld.append(" on transaction ID \\d+\n");
+ bld.append("Recent opcode offsets: (\\d+\\s*){4}$");
try {
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATA_NODES)
.format(false).build();
fail("should not be able to start");
} catch (IOException e) {
assertTrue("error message contains opcodes message",
- e.getMessage().matches(expectedErrorMessage));
+ e.getMessage().matches(bld.toString()));
}
}
Modified: hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/krb5.conf
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/krb5.conf?rev=1295342&r1=1295341&r2=1295342&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/krb5.conf (original)
+++ hadoop/common/branches/HDFS-1623/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/krb5.conf Wed Feb 29 23:47:49 2012
@@ -14,15 +14,24 @@
# 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.
-#
+#
+
[libdefaults]
- default_realm = APACHE.ORG
- udp_preference_limit = 1
- extra_addresses = 127.0.0.1
+ default_realm = EXAMPLE.COM
+ allow_weak_crypto = true
+ default_tkt_enctypes = des-cbc-md5 des-cbc-crc des3-cbc-sha1
+ default_tgs_enctypes = des-cbc-md5 des-cbc-crc des3-cbc-sha1
+
[realms]
- APACHE.ORG = {
- admin_server = localhost:88
- kdc = localhost:88
- }
+ EXAMPLE.COM = {
+ kdc = localhost:60088
+ }
+
[domain_realm]
- localhost = APACHE.ORG
+ .example.com = EXAMPLE.COM
+ example.com = EXAMPLE.COM
+
+[login]
+ krb4_convert = true
+ krb4_get_tickets = false
+
|