소스 검색

HADOOP-9860. Remove class HackedKeytab and HackedKeytabEncoder from hadoop-minikdc once jira DIRSERVER-1882 solved. (ywskycn via tucu)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1515653 13f79535-47bb-0310-9956-ffa450edef68
Alejandro Abdelnur 11 년 전
부모
커밋
540896b39e

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

@@ -32,6 +32,9 @@ Release 2.3.0 - UNRELEASED
     HADOOP-9848. Create a MiniKDC for use with security testing. 
     (ywskycn via tucu)
 
+    HADOOP-9860. Remove class HackedKeytab and HackedKeytabEncoder from 
+    hadoop-minikdc once jira DIRSERVER-1882 solved. (ywskycn via tucu)
+
   OPTIMIZATIONS
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)

+ 1 - 1
hadoop-common-project/hadoop-minikdc/pom.xml

@@ -38,7 +38,7 @@
     <dependency>
       <groupId>org.apache.directory.server</groupId>
       <artifactId>apacheds-all</artifactId>
-      <version>2.0.0-M14</version>
+      <version>2.0.0-M15</version>
       <scope>compile</scope>
     </dependency>
     <dependency>

+ 0 - 42
hadoop-common-project/hadoop-minikdc/src/main/java/org/apache/directory/server/kerberos/shared/keytab/HackedKeytab.java

@@ -1,42 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.directory.server.kerberos.shared.keytab;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-//This is a hack for ApacheDS 2.0.0-M14 to be able to create
-//keytab files with more than one principal.
-//It needs to be in this package because the KeytabEncoder class is package 
-// private.
-//This class can be removed once jira DIRSERVER-1882
-// (https://issues.apache.org/jira/browse/DIRSERVER-1882) solved
-public class HackedKeytab extends Keytab {
-
-  private byte[] keytabVersion = VERSION_52;
-
-  public void write( File file, int principalCount ) throws IOException
-  {
-    HackedKeytabEncoder writer = new HackedKeytabEncoder();
-    ByteBuffer buffer = writer.write( keytabVersion, getEntries(),
-            principalCount );
-    writeFile( buffer, file );
-  }
-
-}

+ 0 - 121
hadoop-common-project/hadoop-minikdc/src/main/java/org/apache/directory/server/kerberos/shared/keytab/HackedKeytabEncoder.java

@@ -1,121 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.directory.server.kerberos.shared.keytab;
-
-import org.apache.directory.shared.kerberos.components.EncryptionKey;
-
-import java.nio.ByteBuffer;
-import java.util.Iterator;
-import java.util.List;
-
-//This is a hack for ApacheDS 2.0.0-M14 to be able to create
-//keytab files with more than one principal.
-//It needs to be in this package because the KeytabEncoder class is package 
-// private.
-//This class can be removed once jira DIRSERVER-1882
-// (https://issues.apache.org/jira/browse/DIRSERVER-1882) solved
-class HackedKeytabEncoder extends KeytabEncoder {
-
-  ByteBuffer write( byte[] keytabVersion, List<KeytabEntry> entries,
-                    int principalCount )
-  {
-    ByteBuffer buffer = ByteBuffer.allocate( 512 * principalCount);
-    putKeytabVersion(buffer, keytabVersion);
-    putKeytabEntries( buffer, entries );
-    buffer.flip();
-    return buffer;
-  }
-
-  private void putKeytabVersion( ByteBuffer buffer, byte[] version )
-  {
-    buffer.put( version );
-  }
-
-  private void putKeytabEntries( ByteBuffer buffer, List<KeytabEntry> entries )
-  {
-    Iterator<KeytabEntry> iterator = entries.iterator();
-
-    while ( iterator.hasNext() )
-    {
-      ByteBuffer entryBuffer = putKeytabEntry( iterator.next() );
-      int size = entryBuffer.position();
-
-      entryBuffer.flip();
-
-      buffer.putInt( size );
-      buffer.put( entryBuffer );
-    }
-  }
-
-  private ByteBuffer putKeytabEntry( KeytabEntry entry )
-  {
-    ByteBuffer buffer = ByteBuffer.allocate( 100 );
-
-    putPrincipalName( buffer, entry.getPrincipalName() );
-
-    buffer.putInt( ( int ) entry.getPrincipalType() );
-
-    buffer.putInt( ( int ) ( entry.getTimeStamp().getTime() / 1000 ) );
-
-    buffer.put( entry.getKeyVersion() );
-
-    putKeyBlock( buffer, entry.getKey() );
-
-    return buffer;
-  }
-
-  private void putPrincipalName( ByteBuffer buffer, String principalName )
-  {
-    String[] split = principalName.split("@");
-    String nameComponent = split[0];
-    String realm = split[1];
-
-    String[] nameComponents = nameComponent.split( "/" );
-
-    // increment for v1
-    buffer.putShort( ( short ) nameComponents.length );
-
-    putCountedString( buffer, realm );
-    // write components
-
-    for ( int ii = 0; ii < nameComponents.length; ii++ )
-    {
-      putCountedString( buffer, nameComponents[ii] );
-    }
-  }
-
-  private void putKeyBlock( ByteBuffer buffer, EncryptionKey key )
-  {
-    buffer.putShort( ( short ) key.getKeyType().getValue() );
-    putCountedBytes( buffer, key.getKeyValue() );
-  }
-
-  private void putCountedString( ByteBuffer buffer, String string )
-  {
-    byte[] data = string.getBytes();
-    buffer.putShort( ( short ) data.length );
-    buffer.put( data );
-  }
-
-  private void putCountedBytes( ByteBuffer buffer, byte[] data )
-  {
-    buffer.putShort( ( short ) data.length );
-    buffer.put( data );
-  }
-
-}

+ 3 - 3
hadoop-common-project/hadoop-minikdc/src/main/java/org/apache/hadoop/minikdc/MiniKdc.java

@@ -37,7 +37,7 @@ import org.apache.directory.server.core.partition.impl.btree.jdbm.JdbmPartition;
 import org.apache.directory.server.core.partition.ldif.LdifPartition;
 import org.apache.directory.server.kerberos.kdc.KdcServer;
 import org.apache.directory.server.kerberos.shared.crypto.encryption.KerberosKeyFactory;
-import org.apache.directory.server.kerberos.shared.keytab.HackedKeytab;
+import org.apache.directory.server.kerberos.shared.keytab.Keytab;
 import org.apache.directory.server.kerberos.shared.keytab.KeytabEntry;
 import org.apache.directory.server.protocol.shared.transport.TcpTransport;
 import org.apache.directory.server.protocol.shared.transport.UdpTransport;
@@ -514,7 +514,7 @@ public class MiniKdc {
   public void createPrincipal(File keytabFile, String ... principals)
           throws Exception {
     String generatedPassword = UUID.randomUUID().toString();
-    HackedKeytab keytab = new HackedKeytab();
+    Keytab keytab = new Keytab();
     List<KeytabEntry> entries = new ArrayList<KeytabEntry>();
     for (String principal : principals) {
       createPrincipal(principal, generatedPassword);
@@ -529,6 +529,6 @@ public class MiniKdc {
       }
     }
     keytab.setEntries(entries);
-    keytab.write(keytabFile, principals.length);
+    keytab.write(keytabFile);
   }
 }

+ 2 - 2
hadoop-common-project/hadoop-minikdc/src/test/java/org/apache/hadoop/minikdc/TestMiniKdc.java

@@ -137,7 +137,7 @@ public class TestMiniKdc extends KerberosSecurityTestcase {
               subject.getPrincipals().iterator().next().getClass());
       Assert.assertEquals(principal + "@" + kdc.getRealm(),
               subject.getPrincipals().iterator().next().getName());
-      loginContext.login();
+      loginContext.logout();
 
       //server login
       subject = new Subject(false, principals, new HashSet<Object>(),
@@ -151,7 +151,7 @@ public class TestMiniKdc extends KerberosSecurityTestcase {
               subject.getPrincipals().iterator().next().getClass());
       Assert.assertEquals(principal + "@" + kdc.getRealm(),
               subject.getPrincipals().iterator().next().getName());
-      loginContext.login();
+      loginContext.logout();
 
     } finally {
       if (loginContext != null) {