You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mina.apache.org by lg...@apache.org on 2015/06/14 09:54:20 UTC

mina-sshd git commit: [SSHD-489] Use an Enum for the KEX negotiation options instead of an integer

Repository: mina-sshd
Updated Branches:
  refs/heads/master 6743fd3e8 -> d44c35824


[SSHD-489] Use an Enum for the KEX negotiation options instead of an integer


Project: http://git-wip-us.apache.org/repos/asf/mina-sshd/repo
Commit: http://git-wip-us.apache.org/repos/asf/mina-sshd/commit/d44c3582
Tree: http://git-wip-us.apache.org/repos/asf/mina-sshd/tree/d44c3582
Diff: http://git-wip-us.apache.org/repos/asf/mina-sshd/diff/d44c3582

Branch: refs/heads/master
Commit: d44c358246d730d0d4c098c113ebf70a1dbd3f0f
Parents: 6743fd3
Author: Lyor Goldstein <lg...@vmware.com>
Authored: Sun Jun 14 10:54:08 2015 +0300
Committer: Lyor Goldstein <lg...@vmware.com>
Committed: Sun Jun 14 10:54:08 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/sshd/client/SshKeyScan.java |   7 +-
 .../sshd/client/session/ClientSessionImpl.java  |  38 +++-
 .../org/apache/sshd/common/SshConstants.java    |  39 ----
 .../sshd/common/cipher/BuiltinCiphers.java      |  22 ++
 .../sshd/common/kex/KexProposalOption.java      | 133 ++++++++++++
 .../sshd/common/session/AbstractSession.java    | 201 ++++++++++++-------
 .../org/apache/sshd/common/session/Session.java |   9 +-
 .../apache/sshd/common/util/ValidateUtils.java  |  14 +-
 .../org/apache/sshd/server/kex/DHGEXServer.java |   8 +-
 .../org/apache/sshd/server/kex/DHGServer.java   |   8 +-
 .../sshd/server/session/ServerSession.java      |  25 ++-
 .../sshd/common/cipher/BuiltinCiphersTest.java  |  16 +-
 .../sshd/common/kex/KexProposalOptionTest.java  | 114 +++++++++++
 .../java/org/apache/sshd/server/ServerTest.java |  16 +-
 14 files changed, 499 insertions(+), 151 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mina-sshd/blob/d44c3582/sshd-core/src/main/java/org/apache/sshd/client/SshKeyScan.java
----------------------------------------------------------------------
diff --git a/sshd-core/src/main/java/org/apache/sshd/client/SshKeyScan.java b/sshd-core/src/main/java/org/apache/sshd/client/SshKeyScan.java
index 4b4a262..8e8baa3 100644
--- a/sshd-core/src/main/java/org/apache/sshd/client/SshKeyScan.java
+++ b/sshd-core/src/main/java/org/apache/sshd/client/SshKeyScan.java
@@ -57,13 +57,13 @@ import java.util.logging.Level;
 import org.apache.sshd.client.future.ConnectFuture;
 import org.apache.sshd.client.session.ClientSession;
 import org.apache.sshd.common.NamedFactory;
-import org.apache.sshd.common.SshConstants;
 import org.apache.sshd.common.SshdSocketAddress;
 import org.apache.sshd.common.cipher.ECCurves;
 import org.apache.sshd.common.config.SshConfigFileReader;
 import org.apache.sshd.common.config.keys.KeyUtils;
 import org.apache.sshd.common.config.keys.PublicKeyEntry;
 import org.apache.sshd.common.io.IoSession;
+import org.apache.sshd.common.kex.KexProposalOption;
 import org.apache.sshd.common.keyprovider.KeyPairProvider;
 import org.apache.sshd.common.session.Session;
 import org.apache.sshd.common.session.SessionListener;
@@ -341,10 +341,9 @@ public class SshKeyScan extends AbstractSimplifiedLog
             IoSession ioSession = session.getIoSession();
             SocketAddress remoteAddress = ioSession.getRemoteAddress();
             String remoteLocation = toString(remoteAddress);
-            for (int paramType = 0; paramType < SshConstants.PROPOSAL_KEX_NAMES.size(); paramType++) {
-                String paramName = SshConstants.PROPOSAL_KEX_NAMES.get(paramType);
+            for (KexProposalOption paramType : KexProposalOption.VALUES) {
                 String paramValue = session.getNegotiatedKexParameter(paramType);
-                log(Level.FINEST, remoteLocation + "[" + paramName + "]: " + paramValue);
+                log(Level.FINEST, remoteLocation + "[" + paramType.getDescription() + "]: " + paramValue);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/mina-sshd/blob/d44c3582/sshd-core/src/main/java/org/apache/sshd/client/session/ClientSessionImpl.java
----------------------------------------------------------------------
diff --git a/sshd-core/src/main/java/org/apache/sshd/client/session/ClientSessionImpl.java b/sshd-core/src/main/java/org/apache/sshd/client/session/ClientSessionImpl.java
index ce7fc7e..b8a143c 100644
--- a/sshd-core/src/main/java/org/apache/sshd/client/session/ClientSessionImpl.java
+++ b/sshd-core/src/main/java/org/apache/sshd/client/session/ClientSessionImpl.java
@@ -51,11 +51,13 @@ import org.apache.sshd.common.ServiceFactory;
 import org.apache.sshd.common.SshConstants;
 import org.apache.sshd.common.SshException;
 import org.apache.sshd.common.SshdSocketAddress;
+import org.apache.sshd.common.cipher.BuiltinCiphers;
 import org.apache.sshd.common.cipher.CipherNone;
 import org.apache.sshd.common.config.keys.KeyUtils;
 import org.apache.sshd.common.future.DefaultSshFuture;
 import org.apache.sshd.common.future.SshFuture;
 import org.apache.sshd.common.io.IoSession;
+import org.apache.sshd.common.kex.KexProposalOption;
 import org.apache.sshd.common.scp.ScpTransferEventListener;
 import org.apache.sshd.common.session.AbstractConnectionService;
 import org.apache.sshd.common.session.AbstractSession;
@@ -270,16 +272,24 @@ public class ClientSessionImpl extends AbstractSession implements ClientSession
         }
         if (kexState.compareAndSet(KEX_STATE_DONE, KEX_STATE_INIT)) {
             reexchangeFuture = new DefaultSshFuture(null);
-            if (!serverProposal[SshConstants.PROPOSAL_ENC_ALGS_CTOS].matches("(^|.*,)none($|,.*)")
-                    || !serverProposal[SshConstants.PROPOSAL_ENC_ALGS_STOC].matches("(^|.*,)none($|,.*)")) {
+            
+            String c2sEncServer = serverProposal.get(KexProposalOption.C2SENC);
+            boolean c2sEncServerNone = BuiltinCiphers.Constants.isNoneCipherIncluded(c2sEncServer);
+            String s2cEncServer = serverProposal.get(KexProposalOption.S2CENC);
+            boolean s2cEncServerNone = BuiltinCiphers.Constants.isNoneCipherIncluded(s2cEncServer);
+            String c2sEncClient = clientProposal.get(KexProposalOption.C2SENC);
+            boolean c2sEncClientNone = BuiltinCiphers.Constants.isNoneCipherIncluded(c2sEncClient);
+            String s2cEncClient = clientProposal.get(KexProposalOption.S2CENC);
+            boolean s2cEncClientNone = BuiltinCiphers.Constants.isNoneCipherIncluded(s2cEncClient);
+
+            if ((!c2sEncServerNone) || (!s2cEncServerNone)) {
                 reexchangeFuture.setValue(new SshException("Server does not support none cipher"));
-            } else if (!clientProposal[SshConstants.PROPOSAL_ENC_ALGS_CTOS].matches("(^|.*,)none($|,.*)")
-                    || !clientProposal[SshConstants.PROPOSAL_ENC_ALGS_STOC].matches("(^|.*,)none($|,.*)")) {
+            } else if ((!c2sEncClientNone) || (!s2cEncClientNone)) {
                 reexchangeFuture.setValue(new SshException("Client does not support none cipher"));
             } else {
                 log.info("Switching to none cipher");
-                clientProposal[SshConstants.PROPOSAL_ENC_ALGS_CTOS] = "none";
-                clientProposal[SshConstants.PROPOSAL_ENC_ALGS_STOC] = "none";
+                clientProposal.put(KexProposalOption.C2SENC, BuiltinCiphers.Constants.NONE);
+                clientProposal.put(KexProposalOption.S2CENC, BuiltinCiphers.Constants.NONE);
                 I_C = sendKexInit(clientProposal);
             }
             return reexchangeFuture;
@@ -486,13 +496,23 @@ public class ClientSessionImpl extends AbstractSession implements ClientSession
     protected void sendKexInit() throws IOException {
         FactoryManager manager = getFactoryManager();
         String algs = NamedResource.Utils.getNames(manager.getSignatureFactories());
-        clientProposal = createProposal(algs);
-        I_C = sendKexInit(clientProposal);
+        Map<KexProposalOption,String> proposal = createProposal(algs);
+        synchronized(clientProposal) {
+            if (!clientProposal.isEmpty()) {
+                clientProposal.clear(); // debug breakpoint
+            }
+            
+            clientProposal.putAll(proposal);
+        }
+
+        I_C = sendKexInit(proposal);
     }
 
     @Override
     protected void receiveKexInit(Buffer buffer) throws IOException {
-        serverProposal = new String[SshConstants.PROPOSAL_MAX];
+        if (!serverProposal.isEmpty()) {
+            serverProposal.clear(); // debug breakpoint
+        }
         I_S = receiveKexInit(buffer, serverProposal);
     }
 

http://git-wip-us.apache.org/repos/asf/mina-sshd/blob/d44c3582/sshd-core/src/main/java/org/apache/sshd/common/SshConstants.java
----------------------------------------------------------------------
diff --git a/sshd-core/src/main/java/org/apache/sshd/common/SshConstants.java b/sshd-core/src/main/java/org/apache/sshd/common/SshConstants.java
index d6a41be..91ce971 100644
--- a/sshd-core/src/main/java/org/apache/sshd/common/SshConstants.java
+++ b/sshd-core/src/main/java/org/apache/sshd/common/SshConstants.java
@@ -18,9 +18,6 @@
  */
 package org.apache.sshd.common;
 
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
 
 /**
  * This interface defines constants for the SSH protocol.
@@ -84,42 +81,6 @@ public interface SshConstants {
     static final byte SSH_MSG_CHANNEL_FAILURE=               100;
 
     //
-    // Values for the algorithms negotiation
-    //
-
-    static final int PROPOSAL_KEX_ALGS = 0;
-    static final int PROPOSAL_SERVER_HOST_KEY_ALGS = 1;
-    static final int PROPOSAL_ENC_ALGS_CTOS = 2;
-    static final int PROPOSAL_ENC_ALGS_STOC = 3;
-    static final int PROPOSAL_MAC_ALGS_CTOS = 4;
-    static final int PROPOSAL_MAC_ALGS_STOC = 5;
-    static final int PROPOSAL_COMP_ALGS_CTOS = 6;
-    static final int PROPOSAL_COMP_ALGS_STOC = 7;
-    static final int PROPOSAL_LANG_CTOS = 8;
-    static final int PROPOSAL_LANG_STOC = 9;
-    static final int PROPOSAL_MAX = 10;
-
-    /**
-     * User-friendly names for the KEX algorithms negotiation items - the
-     * list index matches the {@code PROPOSAL_XXX} constant
-     * @see <A HREF="http://tools.ietf.org/html/rfc4253#section-7.1">RFC-4253 - section 7.1</A>
-     */
-    static final List<String> PROPOSAL_KEX_NAMES =
-            Collections.unmodifiableList(Arrays.asList(
-                    "kex algorithms",
-                    "server host key algorithms",
-                    "encryption algorithms (client to server)",
-                    "encryption algorithms (server to client)",
-                    "mac algorithms (client to server)",
-                    "mac algorithms (server to client)",
-                    "compression algorithms (client to server)",
-                    "compression algorithms (server to client)",
-                    "languages (client to server)",
-                    "languages (server to client)"
-            ));
-
-
-    //
     // Disconnect error codes
     //
     static final int SSH2_DISCONNECT_HOST_NOT_ALLOWED_TO_CONNECT =     1;

http://git-wip-us.apache.org/repos/asf/mina-sshd/blob/d44c3582/sshd-core/src/main/java/org/apache/sshd/common/cipher/BuiltinCiphers.java
----------------------------------------------------------------------
diff --git a/sshd-core/src/main/java/org/apache/sshd/common/cipher/BuiltinCiphers.java b/sshd-core/src/main/java/org/apache/sshd/common/cipher/BuiltinCiphers.java
index ddfe4ce..810b2b2 100644
--- a/sshd-core/src/main/java/org/apache/sshd/common/cipher/BuiltinCiphers.java
+++ b/sshd-core/src/main/java/org/apache/sshd/common/cipher/BuiltinCiphers.java
@@ -29,6 +29,8 @@ import java.util.Map;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 import org.apache.sshd.common.NamedFactory;
 import org.apache.sshd.common.NamedResource;
@@ -338,6 +340,26 @@ public enum BuiltinCiphers implements CipherFactory {
 
     public static final class Constants {
         public static final String NONE = "none";
+        public static final Pattern NONE_CIPHER_PATTERN =
+                Pattern.compile("(^|.*,)" + NONE + "($|,.*)");
+
+        /**
+         * @param s A comma-separated list of ciphers - ignored if {@code null}/empty
+         * @return {@code true} if the {@link #NONE} cipher name appears in it
+         */
+        public static final boolean isNoneCipherIncluded(String s) {
+            if (GenericUtils.isEmpty(s)) {
+                return false;
+            }
+            
+            Matcher m = NONE_CIPHER_PATTERN.matcher(s);
+            if (m.matches()) {
+                return true;
+            } else {
+                return false;   // debug breakpoint
+            }
+        }
+
         public static final String AES128_CBC = "aes128-cbc";
         public static final String AES128_CTR = "aes128-ctr";
         public static final String AES192_CBC = "aes192-cbc";

http://git-wip-us.apache.org/repos/asf/mina-sshd/blob/d44c3582/sshd-core/src/main/java/org/apache/sshd/common/kex/KexProposalOption.java
----------------------------------------------------------------------
diff --git a/sshd-core/src/main/java/org/apache/sshd/common/kex/KexProposalOption.java b/sshd-core/src/main/java/org/apache/sshd/common/kex/KexProposalOption.java
new file mode 100644
index 0000000..31fa8c2
--- /dev/null
+++ b/sshd-core/src/main/java/org/apache/sshd/common/kex/KexProposalOption.java
@@ -0,0 +1,133 @@
+/*
+ * 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.sshd.common.kex;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.List;
+
+import org.apache.sshd.common.util.GenericUtils;
+
+/**
+ * @author <a href="mailto:dev@mina.apache.org">Apache MINA SSHD Project</a>
+ */
+public enum KexProposalOption {
+    ALGORITHMS(Constants.PROPOSAL_KEX_ALGS, "kex algorithms"),
+    SERVERKEYS(Constants.PROPOSAL_SERVER_HOST_KEY_ALGS, "server host key algorithms"),
+    C2SENC(Constants.PROPOSAL_ENC_ALGS_CTOS, "encryption algorithms (client to server)"),
+    S2CENC(Constants.PROPOSAL_ENC_ALGS_STOC, "encryption algorithms (server to client)"),
+    C2SMAC(Constants.PROPOSAL_MAC_ALGS_CTOS, "mac algorithms (client to server)"),
+    S2CMAC(Constants.PROPOSAL_MAC_ALGS_STOC, "mac algorithms (server to client)"),
+    C2SCOMP(Constants.PROPOSAL_COMP_ALGS_CTOS, "compression algorithms (client to server)"),
+    S2CCOMP(Constants.PROPOSAL_COMP_ALGS_STOC, "compression algorithms (server to client)"),
+    C2SLANG(Constants.PROPOSAL_LANG_CTOS, "languages (client to server)"),
+    S2CLANG(Constants.PROPOSAL_LANG_STOC, "languages (server to client)");
+
+    private final int proposalIndex;
+    /**
+     * @return The proposal option location in the KEX array
+     */
+    public final int getProposalIndex() {
+        return proposalIndex;
+    }
+
+    private final String description;
+    /**
+     * @return User-friendly name for the KEX negotiation item
+     * @see <A HREF="http://tools.ietf.org/html/rfc4253#section-7.1">RFC-4253 - section 7.1</A>
+     */
+    public final String getDescription() {
+        return description;
+    }
+
+    KexProposalOption(int index, String desc) {
+        proposalIndex = index;
+        description = desc;
+    }
+
+    /**
+     * Compares values according to {@link KexProposalOption#getProposalIndex()}
+     */
+    public static final Comparator<KexProposalOption> BY_PROPOSAL_INDEX = 
+            new Comparator<KexProposalOption>() {
+                @Override
+                public int compare(KexProposalOption o1, KexProposalOption o2) {
+                    int i1 = (o1 == null) ? (-1) : o1.getProposalIndex();
+                    int i2 = (o2 == null) ? (-1) : o2.getProposalIndex();
+                    return Integer.compare(i1, i2);
+                }
+            };
+    
+    /**
+     * A {@link List} of all the options <U>sorted</U> according to {@link #getProposalIndex(){
+     * @see #BY_PROPOSAL_INDEX
+     */
+    public static final List<KexProposalOption> VALUES = 
+            Collections.unmodifiableList(new ArrayList<KexProposalOption>(EnumSet.allOf(KexProposalOption.class)) {
+                private static final long serialVersionUID = 1L;    // we're not serializing it
+                
+                {
+                    Collections.sort(this, BY_PROPOSAL_INDEX);
+                }
+            });
+    public static final int PROPOSAL_MAX = VALUES.size();
+
+    /**
+     * @param n The option name - ignored if {@code null}/empty
+     * @return The matching {@link KexProposalOption#name()} - case <U>insensitive</U>
+     * or {@code null} if no match found
+     */
+    public static KexProposalOption fromName(String n) {
+        if (GenericUtils.isEmpty(n)) {
+            return null;
+        }
+        
+        for (KexProposalOption o : VALUES) {
+            if (n.equalsIgnoreCase(o.name())) {
+                return o;
+            }
+        }
+        
+        return null;
+    }
+
+    public static KexProposalOption fromProposalIndex(int index) {
+        if ((index < 0) || (index >= VALUES.size())) {
+            return null;
+        } else {
+            return VALUES.get(index);
+        }
+    }
+
+    public static final class Constants {
+        public static final int PROPOSAL_KEX_ALGS = 0;
+        public static final int PROPOSAL_SERVER_HOST_KEY_ALGS = 1;
+        public static final int PROPOSAL_ENC_ALGS_CTOS = 2;
+        public static final int PROPOSAL_ENC_ALGS_STOC = 3;
+        public static final int PROPOSAL_MAC_ALGS_CTOS = 4;
+        public static final int PROPOSAL_MAC_ALGS_STOC = 5;
+        public static final int PROPOSAL_COMP_ALGS_CTOS = 6;
+        public static final int PROPOSAL_COMP_ALGS_STOC = 7;
+        public static final int PROPOSAL_LANG_CTOS = 8;
+        public static final int PROPOSAL_LANG_STOC = 9;
+    }
+}

http://git-wip-us.apache.org/repos/asf/mina-sshd/blob/d44c3582/sshd-core/src/main/java/org/apache/sshd/common/session/AbstractSession.java
----------------------------------------------------------------------
diff --git a/sshd-core/src/main/java/org/apache/sshd/common/session/AbstractSession.java b/sshd-core/src/main/java/org/apache/sshd/common/session/AbstractSession.java
index efd7222..77bace2 100644
--- a/sshd-core/src/main/java/org/apache/sshd/common/session/AbstractSession.java
+++ b/sshd-core/src/main/java/org/apache/sshd/common/session/AbstractSession.java
@@ -29,6 +29,7 @@ import static org.apache.sshd.common.SshConstants.SSH_MSG_UNIMPLEMENTED;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;
+import java.util.EnumMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -57,6 +58,7 @@ import org.apache.sshd.common.future.SshFuture;
 import org.apache.sshd.common.future.SshFutureListener;
 import org.apache.sshd.common.io.IoSession;
 import org.apache.sshd.common.io.IoWriteFuture;
+import org.apache.sshd.common.kex.KexProposalOption;
 import org.apache.sshd.common.kex.KeyExchange;
 import org.apache.sshd.common.mac.Mac;
 import org.apache.sshd.common.random.Random;
@@ -118,9 +120,10 @@ public abstract class AbstractSession extends CloseableUtils.AbstractInnerClosea
     protected byte[] sessionId;
     protected String serverVersion;
     protected String clientVersion;
-    protected String[] serverProposal;
-    protected String[] clientProposal;
-    protected String[] negotiated;
+    // if empty then means not-initialized
+    protected final Map<KexProposalOption,String> serverProposal = new EnumMap<KexProposalOption, String>(KexProposalOption.class);
+    protected final Map<KexProposalOption,String> clientProposal = new EnumMap<KexProposalOption, String>(KexProposalOption.class);
+    private final Map<KexProposalOption,String> negotiationResult  = new EnumMap<KexProposalOption, String>(KexProposalOption.class);
     protected byte[] I_C; // the payload of the client's SSH_MSG_KEXINIT
     protected byte[] I_S; // the payload of the factoryManager's SSH_MSG_KEXINIT
     protected KeyExchange kex;
@@ -271,11 +274,11 @@ public abstract class AbstractSession extends CloseableUtils.AbstractInnerClosea
     }
 
     @Override
-    public String getNegotiatedKexParameter(int paramType) {
-    	if ((paramType < 0) || (negotiated == null) || (paramType >= negotiated.length)) {
+    public String getNegotiatedKexParameter(KexProposalOption paramType) {
+    	if (paramType == null) {
     		return null;
     	} else {
-    		return negotiated[paramType];
+    		return negotiationResult.get(paramType);
     	}
     }
 
@@ -400,9 +403,16 @@ public abstract class AbstractSession extends CloseableUtils.AbstractInnerClosea
                 } else if (!kexState.compareAndSet(KEX_STATE_INIT, KEX_STATE_RUN)) {
                     throw new IllegalStateException("Received SSH_MSG_KEXINIT while key exchange is running");
                 }
-                negotiate();
-                kex = NamedFactory.Utils.create(factoryManager.getKeyExchangeFactories(), negotiated[SshConstants.PROPOSAL_KEX_ALGS]);
-                kex.init(this, serverVersion.getBytes(), clientVersion.getBytes(), I_S, I_C);
+
+                {
+                    Map<KexProposalOption,String> result = negotiate();
+                    String kexAlgorithm = result.get(KexProposalOption.ALGORITHMS);
+                    kex = ValidateUtils.checkNotNull(NamedFactory.Utils.create(factoryManager.getKeyExchangeFactories(), kexAlgorithm),
+                                                     "Unknown negotiated KEX algorithm: %s",
+                                                     kexAlgorithm);
+                    kex.init(this, serverVersion.getBytes(), clientVersion.getBytes(), I_S, I_C);
+                }
+
                 sendEvent(SessionListener.Event.KexCompleted);
                 break;
             case SSH_MSG_NEWKEYS:
@@ -891,21 +901,34 @@ public abstract class AbstractSession extends CloseableUtils.AbstractInnerClosea
      * Create our proposal for SSH negotiation
      *
      * @param hostKeyTypes the list of supported host key types
-     * @return an array of 10 strings holding this proposal
+     * @return The proposal {@link Map>
      */
-    protected String[] createProposal(String hostKeyTypes) {
-        return new String[] {
-                NamedResource.Utils.getNames(factoryManager.getKeyExchangeFactories()),
-                hostKeyTypes,
-                NamedResource.Utils.getNames(factoryManager.getCipherFactories()),
-                NamedResource.Utils.getNames(factoryManager.getCipherFactories()),
-                NamedResource.Utils.getNames(factoryManager.getMacFactories()),
-                NamedResource.Utils.getNames(factoryManager.getMacFactories()),
-                NamedResource.Utils.getNames(factoryManager.getCompressionFactories()),
-                NamedResource.Utils.getNames(factoryManager.getCompressionFactories()),
-                "",
-                ""
-        };
+    protected Map<KexProposalOption,String> createProposal(String hostKeyTypes) {
+        Map<KexProposalOption,String> proposal = new EnumMap<KexProposalOption, String>(KexProposalOption.class);
+        proposal.put(KexProposalOption.ALGORITHMS, NamedResource.Utils.getNames(factoryManager.getKeyExchangeFactories()));
+        proposal.put(KexProposalOption.SERVERKEYS, hostKeyTypes);
+        
+        {
+            String value = NamedResource.Utils.getNames(factoryManager.getCipherFactories());
+            proposal.put(KexProposalOption.S2CENC, value);
+            proposal.put(KexProposalOption.C2SENC, value);
+        }
+
+        {
+            String value = NamedResource.Utils.getNames(factoryManager.getMacFactories());
+            proposal.put(KexProposalOption.S2CMAC, value);
+            proposal.put(KexProposalOption.C2SMAC, value);
+        }
+        
+        {
+            String value = NamedResource.Utils.getNames(factoryManager.getCompressionFactories());
+            proposal.put(KexProposalOption.S2CCOMP, value);
+            proposal.put(KexProposalOption.C2SCOMP, value);
+        }
+        
+        proposal.put(KexProposalOption.S2CLANG, "");
+        proposal.put(KexProposalOption.C2SLANG, "");
+        return proposal;
     }
 
     /**
@@ -916,15 +939,20 @@ public abstract class AbstractSession extends CloseableUtils.AbstractInnerClosea
      * @return the sent packet which must be kept for later use
      * @throws IOException if an error occurred sending the packet
      */
-    protected byte[] sendKexInit(String[] proposal) throws IOException {
+    protected byte[] sendKexInit(Map<KexProposalOption,String> proposal) throws IOException {
         log.debug("Send SSH_MSG_KEXINIT");
         Buffer buffer = createBuffer(SshConstants.SSH_MSG_KEXINIT);
         int p = buffer.wpos();
         buffer.wpos(p + 16);
         random.fill(buffer.array(), p, 16);
-        for (String s : proposal) {
-            buffer.putString(s);
+        for (KexProposalOption paramType : KexProposalOption.VALUES) {
+            String s = proposal.get(paramType);
+            if (log.isTraceEnabled()) {
+                log.trace("sendKexInit(" + paramType.getDescription() + ") " + s);
+            }
+            buffer.putString(GenericUtils.trimToEmpty(s));
         }
+
         buffer.putByte((byte) 0);
         buffer.putInt(0);
         byte[] data = buffer.getCompactData();
@@ -940,7 +968,7 @@ public abstract class AbstractSession extends CloseableUtils.AbstractInnerClosea
      * @param proposal the remote proposal to fill
      * @return the packet data
      */
-    protected byte[] receiveKexInit(Buffer buffer, String[] proposal) {
+    protected byte[] receiveKexInit(Buffer buffer, Map<KexProposalOption,String> proposal) {
         // Recreate the packet payload which will be needed at a later time
         int size = 22;
         byte[] d = buffer.array();
@@ -950,10 +978,15 @@ public abstract class AbstractSession extends CloseableUtils.AbstractInnerClosea
         // Skip 16 bytes of random data
         buffer.rpos(buffer.rpos() + 16);
         // Read proposal
-        for (int i = 0; i < proposal.length; i++) {
-            size += 4;
-            proposal[i] = buffer.getString();
-            size += proposal[i].length();
+        for (KexProposalOption paramType : KexProposalOption.VALUES) {
+            int lastPos = buffer.rpos();
+            String value = buffer.getString();
+            if (log.isTraceEnabled()) {
+                log.trace("receiveKexInit(" + paramType.getDescription() + ") " + value);
+            }
+            int curPos = buffer.rpos(), readLen = curPos - lastPos;
+            proposal.put(paramType, value);
+            size += readLen;
         }
         // Skip 5 bytes
         buffer.getByte();
@@ -1036,24 +1069,43 @@ public abstract class AbstractSession extends CloseableUtils.AbstractInnerClosea
         hash.update(buf, 0, pos);
         MACs2c = hash.digest();
 
-        s2ccipher = NamedFactory.Utils.create(factoryManager.getCipherFactories(), negotiated[SshConstants.PROPOSAL_ENC_ALGS_STOC]);
-        Es2c = resizeKey(Es2c, s2ccipher.getBlockSize(), hash, K, H);
-        s2ccipher.init(isServer ? Cipher.Mode.Encrypt : Cipher.Mode.Decrypt, Es2c, IVs2c);
+        {
+            String value = getNegotiatedKexParameter(KexProposalOption.S2CENC);
+            s2ccipher = ValidateUtils.checkNotNull(NamedFactory.Utils.create(factoryManager.getCipherFactories(), value), "Unknown s2c cipher: %s", value);
+            Es2c = resizeKey(Es2c, s2ccipher.getBlockSize(), hash, K, H);
+            s2ccipher.init(isServer ? Cipher.Mode.Encrypt : Cipher.Mode.Decrypt, Es2c, IVs2c);
+        }
 
-        s2cmac = NamedFactory.Utils.create(factoryManager.getMacFactories(), negotiated[SshConstants.PROPOSAL_MAC_ALGS_STOC]);
-        MACs2c = resizeKey(MACs2c, s2cmac.getBlockSize(), hash, K, H);
-        s2cmac.init(MACs2c);
+        {
+            String value = getNegotiatedKexParameter(KexProposalOption.S2CMAC);
+            s2cmac = ValidateUtils.checkNotNull(NamedFactory.Utils.create(factoryManager.getMacFactories(), value), "Unknown s2c mac: %s", value);
+            MACs2c = resizeKey(MACs2c, s2cmac.getBlockSize(), hash, K, H);
+            s2cmac.init(MACs2c);
+        }
+
+        {
+            String value = getNegotiatedKexParameter(KexProposalOption.S2CCOMP);
+            s2ccomp = NamedFactory.Utils.create(factoryManager.getCompressionFactories(), value);
+        }
 
-        c2scipher = NamedFactory.Utils.create(factoryManager.getCipherFactories(), negotiated[SshConstants.PROPOSAL_ENC_ALGS_CTOS]);
-        Ec2s = resizeKey(Ec2s, c2scipher.getBlockSize(), hash, K, H);
-        c2scipher.init(isServer ? Cipher.Mode.Decrypt : Cipher.Mode.Encrypt, Ec2s, IVc2s);
+        {
+            String value = getNegotiatedKexParameter(KexProposalOption.C2SENC);
+            c2scipher = ValidateUtils.checkNotNull(NamedFactory.Utils.create(factoryManager.getCipherFactories(), value), "Unknown c2s cipher: %s", value);
+            Ec2s = resizeKey(Ec2s, c2scipher.getBlockSize(), hash, K, H);
+            c2scipher.init(isServer ? Cipher.Mode.Decrypt : Cipher.Mode.Encrypt, Ec2s, IVc2s);
+        }
 
-        c2smac = NamedFactory.Utils.create(factoryManager.getMacFactories(), negotiated[SshConstants.PROPOSAL_MAC_ALGS_CTOS]);
-        MACc2s = resizeKey(MACc2s, c2smac.getBlockSize(), hash, K, H);
-        c2smac.init(MACc2s);
+        {
+            String value = getNegotiatedKexParameter(KexProposalOption.C2SMAC);
+            c2smac = ValidateUtils.checkNotNull(NamedFactory.Utils.create(factoryManager.getMacFactories(), value), "Unknown c2s mac: %s", value);
+            MACc2s = resizeKey(MACc2s, c2smac.getBlockSize(), hash, K, H);
+            c2smac.init(MACc2s);
+        }
 
-        s2ccomp = NamedFactory.Utils.create(factoryManager.getCompressionFactories(), negotiated[SshConstants.PROPOSAL_COMP_ALGS_STOC]);
-        c2scomp = NamedFactory.Utils.create(factoryManager.getCompressionFactories(), negotiated[SshConstants.PROPOSAL_COMP_ALGS_CTOS]);
+        {
+            String value = getNegotiatedKexParameter(KexProposalOption.C2SCOMP);
+            c2scomp = NamedFactory.Utils.create(factoryManager.getCompressionFactories(), value);
+        }
 
         if (isServer) {
             outCipher = s2ccipher;
@@ -1154,60 +1206,71 @@ public abstract class AbstractSession extends CloseableUtils.AbstractInnerClosea
 
     /**
      * Compute the negotiated proposals by merging the client and
-     * server proposal.  The negotiated proposal will be stored in
+     * server proposal. The negotiated proposal will also be stored in
      * the {@link #negotiated} property.
+     * @return The negotiated options
      */
-    protected void negotiate() {
-        String[] guess = new String[SshConstants.PROPOSAL_MAX];
-        for (int i = 0; i < SshConstants.PROPOSAL_MAX; i++) {
-        	String paramName = SshConstants.PROPOSAL_KEX_NAMES.get(i);
-        	String clientParamValue = clientProposal[i];
-        	String serverParamValue = serverProposal[i];
+    protected Map<KexProposalOption,String> negotiate() {
+        Map<KexProposalOption,String> guess = new EnumMap<KexProposalOption, String>(KexProposalOption.class);
+        for (KexProposalOption paramType : KexProposalOption.VALUES) {
+        	String clientParamValue = clientProposal.get(paramType);
+        	String serverParamValue = serverProposal.get(paramType);
             String[] c = GenericUtils.split(clientParamValue, ',');
             String[] s = GenericUtils.split(serverParamValue, ',');
             for (String ci : c) {
                 for (String si : s) {
                     if (ci.equals(si)) {
-                        guess[i] = ci;
+                        guess.put(paramType, ci);
                         break;
                     }
                 }
-                if (guess[i] != null) {
+                
+                String value = guess.get(paramType);
+                if (value != null) {
                     break;
                 }
             }
             
             // check if reached an agreement
-            if (guess[i] == null) {
-            	String	message="Unable to negotiate key exchange for " + paramName
+            String value = guess.get(paramType);
+            if (value == null) {
+            	String	message="Unable to negotiate key exchange for " + paramType.getDescription()
             				  + " (client: " + clientParamValue + " / server: " + serverParamValue + ")";
                 // OK if could not negotiate languages
-            	if ((i != SshConstants.PROPOSAL_LANG_CTOS) && (i != SshConstants.PROPOSAL_LANG_STOC)) {
-            		throw new IllegalStateException(message);
+            	if (KexProposalOption.S2CLANG.equals(paramType) || KexProposalOption.C2SLANG.equals(paramType)) {
+                    if (log.isTraceEnabled()) {
+                        log.trace(message);
+                    }
             	} else {
-            		if (log.isTraceEnabled()) {
-            			log.trace(message);
-            		}
+            		throw new IllegalStateException(message);
             	}
             } else {
             	if (log.isTraceEnabled()) {
-            		log.trace("Kex: negotiate(" + paramName + ") guess=" + guess[i]
+            		log.trace("Kex: negotiate(" +  paramType.getDescription() + ") guess=" + value
             				+ " (client: " + clientParamValue + " / server: " + serverParamValue);
             	}
             }
         }
-        negotiated = guess;
+        
+        synchronized(negotiationResult) {
+            if (!negotiationResult.isEmpty()) {
+                negotiationResult.clear(); // debug breakpoint
+            }
+            negotiationResult.putAll(guess);
+        }
 
         if (log.isDebugEnabled()) {
             log.debug("Kex: server->client {} {} {}",
-                      new Object[]{negotiated[SshConstants.PROPOSAL_ENC_ALGS_STOC],
-                            negotiated[SshConstants.PROPOSAL_MAC_ALGS_STOC],
-                            negotiated[SshConstants.PROPOSAL_COMP_ALGS_STOC]});
+                      guess.get(KexProposalOption.S2CENC),
+                      guess.get(KexProposalOption.S2CMAC),
+                      guess.get(KexProposalOption.S2CCOMP));
             log.debug("Kex: client->server {} {} {}",
-                      new Object[]{negotiated[SshConstants.PROPOSAL_ENC_ALGS_CTOS],
-                            negotiated[SshConstants.PROPOSAL_MAC_ALGS_CTOS],
-                            negotiated[SshConstants.PROPOSAL_COMP_ALGS_CTOS]});
+                      guess.get(KexProposalOption.C2SENC),
+                      guess.get(KexProposalOption.C2SMAC),
+                      guess.get(KexProposalOption.C2SCOMP));
         }
+        
+        return guess;
     }
 
     protected void requestSuccess(Buffer buffer) throws Exception{

http://git-wip-us.apache.org/repos/asf/mina-sshd/blob/d44c3582/sshd-core/src/main/java/org/apache/sshd/common/session/Session.java
----------------------------------------------------------------------
diff --git a/sshd-core/src/main/java/org/apache/sshd/common/session/Session.java b/sshd-core/src/main/java/org/apache/sshd/common/session/Session.java
index ecae304..6bda1ec 100644
--- a/sshd-core/src/main/java/org/apache/sshd/common/session/Session.java
+++ b/sshd-core/src/main/java/org/apache/sshd/common/session/Session.java
@@ -27,6 +27,7 @@ import org.apache.sshd.common.Service;
 import org.apache.sshd.common.future.SshFuture;
 import org.apache.sshd.common.io.IoSession;
 import org.apache.sshd.common.io.IoWriteFuture;
+import org.apache.sshd.common.kex.KexProposalOption;
 import org.apache.sshd.common.util.buffer.Buffer;
 
 /**
@@ -93,12 +94,12 @@ public interface Session extends Closeable {
 
     /**
      * Retrieve one of the negotiated values during the KEX stage
-     * @param paramType The parameter type index - one of the {@link SSHConstants}
-     *  {@code PROPOSAL_XXX} values
+     * @param paramType The request {@link KexProposalOption} value - ignored
+     * if {@code null} 
      * @return The negotiated parameter value - {@code null} if invalid
-     * parameter index or no negotiated value
+     * parameter or no negotiated value
      */
-    String getNegotiatedKexParameter(int paramType);
+    String getNegotiatedKexParameter(KexProposalOption paramType);
 
     /**
      * Retrieve a configuration property as an integer

http://git-wip-us.apache.org/repos/asf/mina-sshd/blob/d44c3582/sshd-core/src/main/java/org/apache/sshd/common/util/ValidateUtils.java
----------------------------------------------------------------------
diff --git a/sshd-core/src/main/java/org/apache/sshd/common/util/ValidateUtils.java b/sshd-core/src/main/java/org/apache/sshd/common/util/ValidateUtils.java
index 78e1fc1..735ba34 100644
--- a/sshd-core/src/main/java/org/apache/sshd/common/util/ValidateUtils.java
+++ b/sshd-core/src/main/java/org/apache/sshd/common/util/ValidateUtils.java
@@ -30,6 +30,14 @@ public final class ValidateUtils {
         throw new UnsupportedOperationException("No instance");
     }
 
+    public static final <T> T checkNotNull(T t, String message, Object arg) {
+        if (t == null) {
+            throwIllegalArgumentException(message, arg);
+        }
+        
+        return t;
+    }
+
     public static final <T> T checkNotNull(T t, String message, Object ... args) {
         checkTrue(t != null, message, args);
         return t;
@@ -61,7 +69,11 @@ public final class ValidateUtils {
     
     public static final void checkTrue(boolean flag, String message, Object ... args) {
         if (!flag) {
-            throw new IllegalArgumentException(String.format(message, args));
+            throwIllegalArgumentException(message, args);
         }
     }
+    
+    public static final void throwIllegalArgumentException(String message, Object ... args) {
+        throw new IllegalArgumentException(String.format(message, args));
+    }
 }

http://git-wip-us.apache.org/repos/asf/mina-sshd/blob/d44c3582/sshd-core/src/main/java/org/apache/sshd/server/kex/DHGEXServer.java
----------------------------------------------------------------------
diff --git a/sshd-core/src/main/java/org/apache/sshd/server/kex/DHGEXServer.java b/sshd-core/src/main/java/org/apache/sshd/server/kex/DHGEXServer.java
index 5e8376c..85375bd 100644
--- a/sshd-core/src/main/java/org/apache/sshd/server/kex/DHGEXServer.java
+++ b/sshd-core/src/main/java/org/apache/sshd/server/kex/DHGEXServer.java
@@ -27,6 +27,7 @@ import java.security.KeyPair;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.sshd.common.FactoryManager;
 import org.apache.sshd.common.FactoryManagerUtils;
 import org.apache.sshd.common.NamedFactory;
 import org.apache.sshd.common.SshConstants;
@@ -34,12 +35,14 @@ import org.apache.sshd.common.SshException;
 import org.apache.sshd.common.kex.DHFactory;
 import org.apache.sshd.common.kex.DHG;
 import org.apache.sshd.common.kex.DHGroupData;
+import org.apache.sshd.common.kex.KexProposalOption;
 import org.apache.sshd.common.kex.KeyExchange;
 import org.apache.sshd.common.random.Random;
 import org.apache.sshd.common.session.AbstractSession;
 import org.apache.sshd.common.signature.Signature;
 import org.apache.sshd.common.util.GenericUtils;
 import org.apache.sshd.common.util.SecurityUtils;
+import org.apache.sshd.common.util.ValidateUtils;
 import org.apache.sshd.common.util.buffer.Buffer;
 import org.apache.sshd.common.util.buffer.BufferUtils;
 import org.apache.sshd.common.util.buffer.ByteArrayBuffer;
@@ -156,8 +159,9 @@ public class DHGEXServer extends AbstractDHServerKeyExchange {
 
             byte[] K_S;
             KeyPair kp = session.getHostKey();
-            String algo = session.getNegotiated(SshConstants.PROPOSAL_SERVER_HOST_KEY_ALGS);
-            Signature sig = NamedFactory.Utils.create(session.getFactoryManager().getSignatureFactories(), algo);
+            String algo = session.getNegotiatedKexParameter(KexProposalOption.SERVERKEYS);
+            FactoryManager manager = session.getFactoryManager();
+            Signature sig = ValidateUtils.checkNotNull(NamedFactory.Utils.create(manager.getSignatureFactories(), algo), "Unknown negotiated server keys: %s", algo);
             sig.init(kp.getPublic(), kp.getPrivate());
 
             buffer = new ByteArrayBuffer();

http://git-wip-us.apache.org/repos/asf/mina-sshd/blob/d44c3582/sshd-core/src/main/java/org/apache/sshd/server/kex/DHGServer.java
----------------------------------------------------------------------
diff --git a/sshd-core/src/main/java/org/apache/sshd/server/kex/DHGServer.java b/sshd-core/src/main/java/org/apache/sshd/server/kex/DHGServer.java
index 507cea4..61f1a41 100644
--- a/sshd-core/src/main/java/org/apache/sshd/server/kex/DHGServer.java
+++ b/sshd-core/src/main/java/org/apache/sshd/server/kex/DHGServer.java
@@ -20,14 +20,17 @@ package org.apache.sshd.server.kex;
 
 import java.security.KeyPair;
 
+import org.apache.sshd.common.FactoryManager;
 import org.apache.sshd.common.NamedFactory;
 import org.apache.sshd.common.SshConstants;
 import org.apache.sshd.common.SshException;
 import org.apache.sshd.common.kex.AbstractDH;
 import org.apache.sshd.common.kex.DHFactory;
+import org.apache.sshd.common.kex.KexProposalOption;
 import org.apache.sshd.common.kex.KeyExchange;
 import org.apache.sshd.common.session.AbstractSession;
 import org.apache.sshd.common.signature.Signature;
+import org.apache.sshd.common.util.ValidateUtils;
 import org.apache.sshd.common.util.buffer.Buffer;
 import org.apache.sshd.common.util.buffer.BufferUtils;
 import org.apache.sshd.common.util.buffer.ByteArrayBuffer;
@@ -88,8 +91,9 @@ public class DHGServer extends AbstractDHServerKeyExchange {
 
         byte[] K_S;
         KeyPair kp = session.getHostKey();
-        String algo = session.getNegotiated(SshConstants.PROPOSAL_SERVER_HOST_KEY_ALGS);
-        Signature sig = NamedFactory.Utils.create(session.getFactoryManager().getSignatureFactories(), algo);
+        String algo = session.getNegotiatedKexParameter(KexProposalOption.SERVERKEYS);
+        FactoryManager manager = session.getFactoryManager();
+        Signature sig = ValidateUtils.checkNotNull(NamedFactory.Utils.create(manager.getSignatureFactories(), algo), "Unknown negotiated server keys: %s", algo);
         sig.init(kp.getPublic(), kp.getPrivate());
 
         buffer = new ByteArrayBuffer();

http://git-wip-us.apache.org/repos/asf/mina-sshd/blob/d44c3582/sshd-core/src/main/java/org/apache/sshd/server/session/ServerSession.java
----------------------------------------------------------------------
diff --git a/sshd-core/src/main/java/org/apache/sshd/server/session/ServerSession.java b/sshd-core/src/main/java/org/apache/sshd/server/session/ServerSession.java
index 5112105..d67a023 100644
--- a/sshd-core/src/main/java/org/apache/sshd/server/session/ServerSession.java
+++ b/sshd-core/src/main/java/org/apache/sshd/server/session/ServerSession.java
@@ -21,6 +21,7 @@ package org.apache.sshd.server.session;
 import java.io.IOException;
 import java.security.KeyPair;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.sshd.common.FactoryManager;
 import org.apache.sshd.common.FactoryManagerUtils;
@@ -31,6 +32,7 @@ import org.apache.sshd.common.SshException;
 import org.apache.sshd.common.future.SshFutureListener;
 import org.apache.sshd.common.io.IoSession;
 import org.apache.sshd.common.io.IoWriteFuture;
+import org.apache.sshd.common.kex.KexProposalOption;
 import org.apache.sshd.common.keyprovider.KeyPairProvider;
 import org.apache.sshd.common.session.AbstractSession;
 import org.apache.sshd.common.util.GenericUtils;
@@ -60,10 +62,6 @@ public class ServerSession extends AbstractSession {
         sendServerIdentification();
     }
 
-    public String getNegotiated(int index) {
-        return negotiated[index];
-    }
-
     @Override
     public ServerFactoryManager getFactoryManager() {
         return (ServerFactoryManager) factoryManager;
@@ -143,8 +141,16 @@ public class ServerSession extends AbstractSession {
                                    "sendKexInit(" + provided + ") none of the keys appears in supported list: " + supported);
         }
 
-        serverProposal = createProposal(resolvedHostKeys.toString());
-        I_S = sendKexInit(serverProposal);
+        Map<KexProposalOption,String> proposal = createProposal(resolvedHostKeys.toString());
+        synchronized(serverProposal) {
+            if (!serverProposal.isEmpty()) {
+                serverProposal.clear(); // debug breakpoint
+            }
+            
+            serverProposal.putAll(proposal);
+        }
+
+        I_S = sendKexInit(proposal);
     }
 
     @Override
@@ -172,12 +178,15 @@ public class ServerSession extends AbstractSession {
 
     @Override
     protected void receiveKexInit(Buffer buffer) throws IOException {
-        clientProposal = new String[SshConstants.PROPOSAL_MAX];
+        if (!clientProposal.isEmpty()) {
+            clientProposal.clear(); // debug breakpoint
+        }
         I_C = receiveKexInit(buffer, clientProposal);
     }
 
     public KeyPair getHostKey() {
-        return factoryManager.getKeyPairProvider().loadKey(negotiated[SshConstants.PROPOSAL_SERVER_HOST_KEY_ALGS]);
+        String value = getNegotiatedKexParameter(KexProposalOption.SERVERKEYS);
+        return factoryManager.getKeyPairProvider().loadKey(value);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/mina-sshd/blob/d44c3582/sshd-core/src/test/java/org/apache/sshd/common/cipher/BuiltinCiphersTest.java
----------------------------------------------------------------------
diff --git a/sshd-core/src/test/java/org/apache/sshd/common/cipher/BuiltinCiphersTest.java b/sshd-core/src/test/java/org/apache/sshd/common/cipher/BuiltinCiphersTest.java
index 4161b8b..ad1cf02 100644
--- a/sshd-core/src/test/java/org/apache/sshd/common/cipher/BuiltinCiphersTest.java
+++ b/sshd-core/src/test/java/org/apache/sshd/common/cipher/BuiltinCiphersTest.java
@@ -20,12 +20,14 @@
 package org.apache.sshd.common.cipher;
 
 import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.List;
+import java.util.Objects;
 import java.util.Random;
 import java.util.Set;
 
@@ -98,8 +100,18 @@ public class BuiltinCiphersTest extends BaseTestSupport {
         Set<BuiltinCiphers> avail=EnumSet.noneOf(BuiltinCiphers.class);
         Field[]             fields=BuiltinCiphers.Constants.class.getFields();
         for (Field f : fields) {
-            String          name=(String) f.get(null);
-            BuiltinCiphers  value=BuiltinCiphers.fromFactoryName(name);
+            int mods = f.getModifiers();
+            if (!Modifier.isStatic(mods)) {
+                continue;
+            }
+
+            Class<?> type = f.getType();
+            if (!String.class.isAssignableFrom(type)) {
+                continue;
+            }
+
+            String name = Objects.toString(f.get(null));
+            BuiltinCiphers value = BuiltinCiphers.fromFactoryName(name);
             assertNotNull("No match found for " + name, value);
             assertTrue(name + " re-specified", avail.add(value));
         }

http://git-wip-us.apache.org/repos/asf/mina-sshd/blob/d44c3582/sshd-core/src/test/java/org/apache/sshd/common/kex/KexProposalOptionTest.java
----------------------------------------------------------------------
diff --git a/sshd-core/src/test/java/org/apache/sshd/common/kex/KexProposalOptionTest.java b/sshd-core/src/test/java/org/apache/sshd/common/kex/KexProposalOptionTest.java
new file mode 100644
index 0000000..33418ca
--- /dev/null
+++ b/sshd-core/src/test/java/org/apache/sshd/common/kex/KexProposalOptionTest.java
@@ -0,0 +1,114 @@
+/*
+ * 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.sshd.common.kex;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.Collection;
+import java.util.EnumSet;
+
+import org.apache.sshd.common.kex.KexProposalOption.Constants;
+import org.apache.sshd.common.util.GenericUtils;
+import org.apache.sshd.util.BaseTestSupport;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runners.MethodSorters;
+
+/**
+ * @author <a href="mailto:dev@mina.apache.org">Apache MINA SSHD Project</a>
+ */
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class KexProposalOptionTest extends BaseTestSupport {
+    public KexProposalOptionTest() {
+        super();
+    }
+
+    @Test
+    public void testFromName() {
+        for (String n : new String[] { null, "", getCurrentTestName() }) {
+            KexProposalOption o = KexProposalOption.fromName(n);
+            assertNull("Unexpected value for name='" + n + "'", o);
+        }
+
+        for (KexProposalOption expected : KexProposalOption.VALUES) {
+            String n = expected.name();
+            
+            for (int index=0; index < n.length(); index++) {
+                KexProposalOption actual = KexProposalOption.fromName(n);
+                assertSame("Mismatched option for name=" + n, expected, actual);
+                n = shuffleCase(n); // prepare for next iteration
+            }
+        }
+    }
+
+    @Test
+    public void testFromProposalIndex() {
+        for (int index : new int[] { (-1), KexProposalOption.VALUES.size() }) {
+            KexProposalOption o = KexProposalOption.fromProposalIndex(index);
+            assertNull("Unexpected value for index=" + index, o);
+        }
+        
+        for (KexProposalOption expected : KexProposalOption.VALUES) {
+            int index = expected.getProposalIndex();
+            KexProposalOption actual = KexProposalOption.fromProposalIndex(index);
+            assertSame("Mismatched values for index=" + index, expected, actual);
+        }
+    }
+
+    @Test
+    public void testByProposalIndexSortOrder() {
+        for (int index=0; index < KexProposalOption.VALUES.size(); index++) {
+            if (index < 1) {
+                continue;
+            }
+            
+            KexProposalOption o1 = KexProposalOption.VALUES.get(index - 1);
+            KexProposalOption o2 = KexProposalOption.VALUES.get(index);
+            
+            int i1 = o1.getProposalIndex(), i2 = o2.getProposalIndex();
+            assertTrue("Non increasing index for " + o1 + "[" + i1 + "] vs. " + o2 + "[" + i2 + "]", i1 < i2);
+        }
+    }
+
+    @Test
+    public void testAllConstantsCovered() throws Exception {
+        Field[] fields=Constants.class.getFields();
+
+        Collection<KexProposalOption> options = EnumSet.allOf(KexProposalOption.class); 
+        for (Field f : fields) {
+            int mods = f.getModifiers();
+            if (!Modifier.isStatic(mods)) {
+                continue;
+            }
+            
+            Class<?> type = f.getType();
+            if (!Integer.TYPE.isAssignableFrom(type)) {
+                continue;
+            }
+            
+            int index = f.getInt(null);
+            KexProposalOption o = KexProposalOption.fromProposalIndex(index);
+            assertNotNull("No matching option for index=" + index, o);
+            assertTrue("Option not in known options: " + o, options.remove(o));
+        }
+        
+        assertTrue("Not all options covered: " + options, GenericUtils.isEmpty(options));
+    }
+}

http://git-wip-us.apache.org/repos/asf/mina-sshd/blob/d44c3582/sshd-core/src/test/java/org/apache/sshd/server/ServerTest.java
----------------------------------------------------------------------
diff --git a/sshd-core/src/test/java/org/apache/sshd/server/ServerTest.java b/sshd-core/src/test/java/org/apache/sshd/server/ServerTest.java
index 8cc32ea..9ff9398 100644
--- a/sshd-core/src/test/java/org/apache/sshd/server/ServerTest.java
+++ b/sshd-core/src/test/java/org/apache/sshd/server/ServerTest.java
@@ -44,22 +44,16 @@ import org.apache.sshd.client.session.ClientSessionImpl;
 import org.apache.sshd.common.FactoryManager;
 import org.apache.sshd.common.FactoryManagerUtils;
 import org.apache.sshd.common.NamedFactory;
-import org.apache.sshd.common.SshConstants;
 import org.apache.sshd.common.channel.Channel;
 import org.apache.sshd.common.channel.WindowClosedException;
 import org.apache.sshd.common.io.IoSession;
+import org.apache.sshd.common.kex.KexProposalOption;
 import org.apache.sshd.common.session.AbstractConnectionService;
 import org.apache.sshd.common.session.AbstractSession;
 import org.apache.sshd.common.session.Session;
 import org.apache.sshd.common.session.SessionListener;
 import org.apache.sshd.deprecated.ClientUserAuthServiceOld;
 import org.apache.sshd.deprecated.UserAuthPassword;
-import org.apache.sshd.server.Command;
-import org.apache.sshd.server.CommandFactory;
-import org.apache.sshd.server.Environment;
-import org.apache.sshd.server.ExitCallback;
-import org.apache.sshd.server.ServerFactoryManager;
-import org.apache.sshd.server.SshServer;
 import org.apache.sshd.server.command.ScpCommandFactory;
 import org.apache.sshd.server.sftp.SftpSubsystemFactory;
 import org.apache.sshd.util.BaseTestSupport;
@@ -310,10 +304,10 @@ public class ServerTest extends BaseTestSupport {
             protected AbstractSession createSession(IoSession ioSession) throws Exception {
                 return new ClientSessionImpl(client, ioSession) {
                     @Override
-                    protected String[] createProposal(String hostKeyTypes) {
-                        String[] proposal = super.createProposal(hostKeyTypes);
-                        proposal[SshConstants.PROPOSAL_LANG_CTOS] = "en-US";
-                        proposal[SshConstants.PROPOSAL_LANG_STOC] = "en-US";
+                    protected Map<KexProposalOption,String> createProposal(String hostKeyTypes) {
+                        Map<KexProposalOption,String> proposal = super.createProposal(hostKeyTypes);
+                        proposal.put(KexProposalOption.S2CLANG, "en-US");
+                        proposal.put(KexProposalOption.C2SLANG, "en-US");
                         return proposal;
                     }
                 };