Procházet zdrojové kódy

YARN-2936. Changed YARNDelegationTokenIdentifier to set proto fields on getProto method. Contributed by Varun Saxena
(cherry picked from commit 2638f4d0f0da375b0dd08f3188057637ed0f32d5)

Jian He před 10 roky
rodič
revize
fe664b8b8d

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -299,6 +299,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

+ 29 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/YARNDelegationTokenIdentifier.java

@@ -64,19 +64,41 @@ public abstract class YARNDelegationTokenIdentifier extends
     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();
   }
 }

+ 30 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java

@@ -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.NodeId;
 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 class TestYARNTokenIdentifier {
     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 class TestYARNTokenIdentifier {
     
     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