YARN-2936. Changed YARNDelegationTokenIdentifier to set proto fields on getProto method. Contributed by Varun Saxena

This commit is contained in:
Jian He 2015-01-07 16:14:22 -08:00
parent e86943fd64
commit 2638f4d0f0
3 changed files with 62 additions and 8 deletions

View File

@ -331,6 +331,9 @@ Release 2.7.0 - UNRELEASED
YARN-3010. Fixed findbugs warning in AbstractYarnScheduler. (Yi Liu via
jianhe)
YARN-2936. Changed YARNDelegationTokenIdentifier to set proto fields on
getProto method. (Varun Saxena via jianhe)
Release 2.6.0 - 2014-11-18
INCOMPATIBLE CHANGES

View File

@ -64,19 +64,41 @@ public synchronized void readFields(DataInput in) throws IOException {
setMasterKeyId(builder.getMasterKeyId());
}
private void setBuilderFields() {
if (builder.getOwner() != null &&
!builder.getOwner().equals(getOwner().toString())) {
builder.setOwner(getOwner().toString());
}
if (builder.getRenewer() != null &&
!builder.getRenewer().equals(getRenewer().toString())) {
builder.setRenewer(getRenewer().toString());
}
if (builder.getRealUser() != null &&
!builder.getRealUser().equals(getRealUser().toString())) {
builder.setRealUser(getRealUser().toString());
}
if (builder.getIssueDate() != getIssueDate()) {
builder.setIssueDate(getIssueDate());
}
if (builder.getMaxDate() != getMaxDate()) {
builder.setMaxDate(getMaxDate());
}
if (builder.getSequenceNumber() != getSequenceNumber()) {
builder.setSequenceNumber(getSequenceNumber());
}
if (builder.getMasterKeyId() != getMasterKeyId()) {
builder.setMasterKeyId(getMasterKeyId());
}
}
@Override
public synchronized void write(DataOutput out) throws IOException {
builder.setOwner(getOwner().toString());
builder.setRenewer(getRenewer().toString());
builder.setRealUser(getRealUser().toString());
builder.setIssueDate(getIssueDate());
builder.setMaxDate(getMaxDate());
builder.setSequenceNumber(getSequenceNumber());
builder.setMasterKeyId(getMasterKeyId());
setBuilderFields();
builder.build().writeTo((DataOutputStream) out);
}
public YARNDelegationTokenIdentifierProto getProto() {
setBuilderFields();
return builder.build();
}
}

View File

@ -17,6 +17,8 @@
*/
package org.apache.hadoop.yarn.security;
import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
@ -31,6 +33,7 @@
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos.YARNDelegationTokenIdentifierProto;
import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
@ -224,7 +227,7 @@ public void testRMDelegationTokenIdentifier() throws IOException {
DataInputBuffer dib = new DataInputBuffer();
dib.reset(tokenContent, tokenContent.length);
anotherToken.readFields(dib);
dib.close();
// verify the whole record equals with original record
Assert.assertEquals("Token is not the same after serialization " +
"and deserialization.", token, anotherToken);
@ -249,6 +252,32 @@ public void testRMDelegationTokenIdentifier() throws IOException {
Assert.assertEquals("masterKeyId from proto is not the same with original token",
anotherToken.getMasterKeyId(), masterKeyId);
// Test getProto
RMDelegationTokenIdentifier token1 =
new RMDelegationTokenIdentifier(owner, renewer, realUser);
token1.setIssueDate(issueDate);
token1.setMaxDate(maxDate);
token1.setSequenceNumber(sequenceNumber);
token1.setMasterKeyId(masterKeyId);
YARNDelegationTokenIdentifierProto tokenProto = token1.getProto();
// Write token proto to stream
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream out = new DataOutputStream(baos);
tokenProto.writeTo(out);
// Read token
byte[] tokenData = baos.toByteArray();
RMDelegationTokenIdentifier readToken = new RMDelegationTokenIdentifier();
DataInputBuffer db = new DataInputBuffer();
db.reset(tokenData, tokenData.length);
readToken.readFields(db);
// Verify if read token equals with original token
Assert.assertEquals("Token from getProto is not the same after " +
"serialization and deserialization.", token1, readToken);
db.close();
out.close();
}
@Test