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 2020/05/04 15:31:30 UTC

[mina-sshd] 01/02: [SSHD-984] Writing keys in modern OpenSSH format

This is an automated email from the ASF dual-hosted git repository.

lgoldstein pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/mina-sshd.git

commit 42df03db207be7be5479acf97335ea8ae1603c21
Author: Thomas Wolf <th...@paranor.ch>
AuthorDate: Sun May 3 12:37:01 2020 +0200

    [SSHD-984] Writing keys in modern OpenSSH format
    
    Add support for writing keys in the modern OpenSSH key format using
    the OpenBSD bcrypt KDF for writing passphrase-protected, encrypted
    private keys.
    
    Add a new OpenSSHKeyPairResourceWriter using a specialized
    OpenSSHKeyEncryptionContext to make the number of KDF rounds
    configurable and to have the passphrase as a char[] instead of as a
    String. Introduce a new SecureByteArrayOutputStream.
    
    Includes test cases that generate various keys, write them to files,
    and load them again. Since this only tests that Apache MINA sshd can
    work with these keys, I've also manually verified that command-line
    OpenSSH can use keys generated and written by Java to actually connect
    to an SSH server.
---
 .../common/config/keys/PrivateKeyEntryDecoder.java |  18 +-
 .../openssh/OpenSSHDSSPrivateKeyEntryDecoder.java  |   6 +-
 .../OpenSSHECDSAPrivateKeyEntryDecoder.java        |  16 +-
 .../openssh/OpenSSHRSAPrivateKeyDecoder.java       |  18 ++
 .../config/keys/writer/KeyPairResourceWriter.java  |  84 +++++
 .../openssh/OpenSSHKeyEncryptionContext.java       | 147 +++++++++
 .../openssh/OpenSSHKeyPairResourceWriter.java      | 335 +++++++++++++++++++
 .../util/io/SecureByteArrayOutputStream.java       |  61 ++++
 .../OpenSSHEd25519PrivateKeyEntryDecoder.java      |   4 +-
 .../openssh/OpenSSHKeyPairResourceWriterTest.java  | 358 +++++++++++++++++++++
 10 files changed, 1032 insertions(+), 15 deletions(-)

diff --git a/sshd-common/src/main/java/org/apache/sshd/common/config/keys/PrivateKeyEntryDecoder.java b/sshd-common/src/main/java/org/apache/sshd/common/config/keys/PrivateKeyEntryDecoder.java
index 767ac0b..4deccc8 100644
--- a/sshd-common/src/main/java/org/apache/sshd/common/config/keys/PrivateKeyEntryDecoder.java
+++ b/sshd-common/src/main/java/org/apache/sshd/common/config/keys/PrivateKeyEntryDecoder.java
@@ -36,6 +36,7 @@ import org.apache.sshd.common.session.SessionContext;
 import org.apache.sshd.common.util.GenericUtils;
 import org.apache.sshd.common.util.NumberUtils;
 import org.apache.sshd.common.util.ValidateUtils;
+import org.apache.sshd.common.util.io.SecureByteArrayOutputStream;
 
 /**
  * @param  <PUB> Type of {@link PublicKey}
@@ -121,16 +122,19 @@ public interface PrivateKeyEntryDecoder<PUB extends PublicKey, PRV extends Priva
             throws IOException, GeneralSecurityException;
 
     /**
-     * Encodes the {@link PrivateKey} using the {@code OpenSSH} format - same one used by the {@code decodePublicKey}
-     * method(s)
+     * Encodes the {@link PrivateKey} using the {@code OpenSSH} format - same one
+     * used by the {@code decodePublicKey} method(s)
      *
-     * @param  s           The {@link OutputStream} to write the data to
-     * @param  key         The {@link PrivateKey} - may not be {@code null}
-     * @return             The key type value - one of the {@link #getSupportedKeyTypes()} or {@code null} if encoding
-     *                     not supported
+     * @param s      The {@link SecureByteArrayOutputStream} to write the data to.
+     * @param key    The {@link PrivateKey} - may not be {@code null}
+     * @param pubKey The {@link PublicKey} belonging to the private key - must be
+     *               non-{@code null} if {@link #isPublicKeyRecoverySupported()
+     *               public key recovery} is not supported
+     * @return The key type value - one of the {@link #getSupportedKeyTypes()} or
+     *         {@code null} if encoding not supported
      * @throws IOException If failed to generate the encoding
      */
-    default String encodePrivateKey(OutputStream s, PRV key) throws IOException {
+    default String encodePrivateKey(SecureByteArrayOutputStream s, PRV key, PUB pubKey) throws IOException {
         Objects.requireNonNull(key, "No private key provided");
         return null;
     }
diff --git a/sshd-common/src/main/java/org/apache/sshd/common/config/keys/loader/openssh/OpenSSHDSSPrivateKeyEntryDecoder.java b/sshd-common/src/main/java/org/apache/sshd/common/config/keys/loader/openssh/OpenSSHDSSPrivateKeyEntryDecoder.java
index 52e3909..b543ea9 100644
--- a/sshd-common/src/main/java/org/apache/sshd/common/config/keys/loader/openssh/OpenSSHDSSPrivateKeyEntryDecoder.java
+++ b/sshd-common/src/main/java/org/apache/sshd/common/config/keys/loader/openssh/OpenSSHDSSPrivateKeyEntryDecoder.java
@@ -21,7 +21,6 @@ package org.apache.sshd.common.config.keys.loader.openssh;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.OutputStream;
 import java.math.BigInteger;
 import java.security.GeneralSecurityException;
 import java.security.InvalidKeyException;
@@ -42,6 +41,7 @@ import org.apache.sshd.common.config.keys.KeyUtils;
 import org.apache.sshd.common.config.keys.impl.AbstractPrivateKeyEntryDecoder;
 import org.apache.sshd.common.keyprovider.KeyPairProvider;
 import org.apache.sshd.common.session.SessionContext;
+import org.apache.sshd.common.util.io.SecureByteArrayOutputStream;
 import org.apache.sshd.common.util.security.SecurityUtils;
 
 /**
@@ -83,7 +83,7 @@ public class OpenSSHDSSPrivateKeyEntryDecoder extends AbstractPrivateKeyEntryDec
     }
 
     @Override
-    public String encodePrivateKey(OutputStream s, DSAPrivateKey key) throws IOException {
+    public String encodePrivateKey(SecureByteArrayOutputStream s, DSAPrivateKey key, DSAPublicKey pubKey) throws IOException {
         Objects.requireNonNull(key, "No private key provided");
 
         DSAParams keyParams = Objects.requireNonNull(key.getParams(), "No DSA params available");
@@ -95,7 +95,7 @@ public class OpenSSHDSSPrivateKeyEntryDecoder extends AbstractPrivateKeyEntryDec
         KeyEntryResolver.encodeBigInt(s, g);
 
         BigInteger x = key.getX();
-        BigInteger y = g.modPow(x, p);
+        BigInteger y = pubKey != null ? pubKey.getY() : g.modPow(x, p);
         KeyEntryResolver.encodeBigInt(s, y);
         KeyEntryResolver.encodeBigInt(s, x);
         return KeyPairProvider.SSH_DSS;
diff --git a/sshd-common/src/main/java/org/apache/sshd/common/config/keys/loader/openssh/OpenSSHECDSAPrivateKeyEntryDecoder.java b/sshd-common/src/main/java/org/apache/sshd/common/config/keys/loader/openssh/OpenSSHECDSAPrivateKeyEntryDecoder.java
index 90389ad..ee33129 100644
--- a/sshd-common/src/main/java/org/apache/sshd/common/config/keys/loader/openssh/OpenSSHECDSAPrivateKeyEntryDecoder.java
+++ b/sshd-common/src/main/java/org/apache/sshd/common/config/keys/loader/openssh/OpenSSHECDSAPrivateKeyEntryDecoder.java
@@ -21,7 +21,6 @@ package org.apache.sshd.common.config.keys.loader.openssh;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.OutputStream;
 import java.math.BigInteger;
 import java.security.GeneralSecurityException;
 import java.security.InvalidKeyException;
@@ -44,6 +43,7 @@ import org.apache.sshd.common.config.keys.KeyUtils;
 import org.apache.sshd.common.config.keys.impl.AbstractPrivateKeyEntryDecoder;
 import org.apache.sshd.common.config.keys.impl.ECDSAPublicKeyEntryDecoder;
 import org.apache.sshd.common.session.SessionContext;
+import org.apache.sshd.common.util.io.SecureByteArrayOutputStream;
 import org.apache.sshd.common.util.security.SecurityUtils;
 
 /**
@@ -90,9 +90,19 @@ public class OpenSSHECDSAPrivateKeyEntryDecoder extends AbstractPrivateKeyEntryD
     }
 
     @Override
-    public String encodePrivateKey(OutputStream s, ECPrivateKey key) throws IOException {
+    public String encodePrivateKey(SecureByteArrayOutputStream s, ECPrivateKey key, ECPublicKey pubKey) throws IOException {
         Objects.requireNonNull(key, "No private key provided");
-        return null;
+        Objects.requireNonNull(pubKey, "No public key provided");
+        ECCurves curve = ECCurves.fromECKey(key);
+        if (curve == null) {
+            return null;
+        }
+        String curveName = curve.getName();
+        KeyEntryResolver.encodeString(s, curveName);
+        ECCurves.ECPointCompression.UNCOMPRESSED.writeECPoint(s,
+                curveName, pubKey.getW());
+        KeyEntryResolver.encodeBigInt(s, key.getS());
+        return curve.getKeyType();
     }
 
     @Override
diff --git a/sshd-common/src/main/java/org/apache/sshd/common/config/keys/loader/openssh/OpenSSHRSAPrivateKeyDecoder.java b/sshd-common/src/main/java/org/apache/sshd/common/config/keys/loader/openssh/OpenSSHRSAPrivateKeyDecoder.java
index 2f8001e..9e6173b 100644
--- a/sshd-common/src/main/java/org/apache/sshd/common/config/keys/loader/openssh/OpenSSHRSAPrivateKeyDecoder.java
+++ b/sshd-common/src/main/java/org/apache/sshd/common/config/keys/loader/openssh/OpenSSHRSAPrivateKeyDecoder.java
@@ -42,6 +42,7 @@ import org.apache.sshd.common.config.keys.KeyUtils;
 import org.apache.sshd.common.config.keys.impl.AbstractPrivateKeyEntryDecoder;
 import org.apache.sshd.common.keyprovider.KeyPairProvider;
 import org.apache.sshd.common.session.SessionContext;
+import org.apache.sshd.common.util.io.SecureByteArrayOutputStream;
 import org.apache.sshd.common.util.security.SecurityUtils;
 
 /**
@@ -92,6 +93,23 @@ public class OpenSSHRSAPrivateKeyDecoder extends AbstractPrivateKeyEntryDecoder<
     }
 
     @Override
+    public String encodePrivateKey(SecureByteArrayOutputStream s, RSAPrivateKey key, RSAPublicKey pubKey) throws IOException {
+        Objects.requireNonNull(key, "No private key provided");
+        if (key instanceof RSAPrivateCrtKey) {
+            RSAPrivateCrtKey a = (RSAPrivateCrtKey) key;
+            KeyEntryResolver.encodeBigInt(s, a.getModulus()); // n
+            KeyEntryResolver.encodeBigInt(s, a.getPublicExponent()); // e
+            KeyEntryResolver.encodeBigInt(s, a.getPrivateExponent()); // d
+            // CRT coefficient q^-1 mod p
+            KeyEntryResolver.encodeBigInt(s, a.getCrtCoefficient());
+            KeyEntryResolver.encodeBigInt(s, a.getPrimeP()); // p
+            KeyEntryResolver.encodeBigInt(s, a.getPrimeQ()); // q
+            return KeyPairProvider.SSH_RSA;
+        }
+        return null;
+    }
+
+    @Override
     public boolean isPublicKeyRecoverySupported() {
         return true;
     }
diff --git a/sshd-common/src/main/java/org/apache/sshd/common/config/keys/writer/KeyPairResourceWriter.java b/sshd-common/src/main/java/org/apache/sshd/common/config/keys/writer/KeyPairResourceWriter.java
new file mode 100644
index 0000000..c1be690
--- /dev/null
+++ b/sshd-common/src/main/java/org/apache/sshd/common/config/keys/writer/KeyPairResourceWriter.java
@@ -0,0 +1,84 @@
+/*
+ * 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.config.keys.writer;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.security.GeneralSecurityException;
+import java.security.KeyPair;
+import java.security.PublicKey;
+
+import org.apache.sshd.common.config.keys.loader.PrivateKeyEncryptionContext;
+import org.apache.sshd.common.util.io.SecureByteArrayOutputStream;
+
+/**
+ * A {@code KeyPairResourceWriter} can serialize keys to an external
+ * representation.
+ *
+ * @param <OPTIONS> The type of {@link PrivateKeyEncryptionContext} to use with
+ *                  this {@code KeyPairResourceWriter}.
+ */
+public interface KeyPairResourceWriter<OPTIONS extends PrivateKeyEncryptionContext> {
+
+    /**
+     * Writes a serialization of a private key from a given {@link KeyPair} to a
+     * given {@link SecureByteArrayOutputStream}.
+     *
+     * @param key     to write the private key of
+     * @param comment to write with the private key
+     * @param options for writing the key; may be {@code null} if no encryption is
+     *                wanted. The caller is responsible for
+     *                {@link PrivateKeyEncryptionContext#clear() clearing} the
+     *                options when no longer needed. If the passphrase obtained from
+     *                the context is {@code null} or an empty array (length zero or
+     *                containing only whitespace), the key is written unencrypted.
+     * @param out     to write to
+     * @return a byte array containing the serialized private key data
+     * @throws GeneralSecurityException if the key is inconsistent or unknown, or
+     *                                  the encryption specified cannot be applied
+     * @throws IOException              if the key cannot be written
+     */
+    void writePrivateKey(KeyPair key, String comment, OPTIONS options, SecureByteArrayOutputStream out)
+            throws IOException, GeneralSecurityException;
+
+    /**
+     * Writes a serialization of a public key from a given {@link KeyPair} to a
+     * given {@link OutputStream}.
+     *
+     * @param key     to write the public key of
+     * @param comment to write with the public key
+     * @param out     to write to
+     * @throws GeneralSecurityException if the key is unknown
+     * @throws IOException              if the key cannot be written
+     */
+    void writePublicKey(KeyPair key, String comment, OutputStream out) throws IOException, GeneralSecurityException;
+
+    /**
+     * Writes a serialization of a {@link PublicKey} to a given
+     * {@link OutputStream}.
+     *
+     * @param key     to write
+     * @param comment to write with the key
+     * @param out     to write to
+     * @throws GeneralSecurityException if the key is unknown
+     * @throws IOException              if the key cannot be written
+     */
+    void writePublicKey(PublicKey key, String comment, OutputStream out) throws IOException, GeneralSecurityException;
+}
diff --git a/sshd-common/src/main/java/org/apache/sshd/common/config/keys/writer/openssh/OpenSSHKeyEncryptionContext.java b/sshd-common/src/main/java/org/apache/sshd/common/config/keys/writer/openssh/OpenSSHKeyEncryptionContext.java
new file mode 100644
index 0000000..78313a6
--- /dev/null
+++ b/sshd-common/src/main/java/org/apache/sshd/common/config/keys/writer/openssh/OpenSSHKeyEncryptionContext.java
@@ -0,0 +1,147 @@
+/*
+ * 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.config.keys.writer.openssh;
+
+import java.util.Arrays;
+
+import org.apache.sshd.common.config.keys.loader.PrivateKeyEncryptionContext;
+import org.apache.sshd.common.util.ValidateUtils;
+
+/**
+ * A {@link PrivateKeyEncryptionContext} for use with a
+ * {@link OpenSSHKeyPairResourceWriter}.
+ */
+public class OpenSSHKeyEncryptionContext extends PrivateKeyEncryptionContext {
+
+    /** Default number of bcrypt KDF rounds to apply. */
+    public static final int DEFAULT_KDF_ROUNDS = 16;
+
+    private static final String AES = "AES";
+
+    private char[] passphrase;
+
+    private int kdfRounds = DEFAULT_KDF_ROUNDS;
+
+    public OpenSSHKeyEncryptionContext() {
+        super();
+        setCipherMode("CTR"); // Set default to CTR, as in OpenSSH
+    }
+
+    @Override
+    public String getCipherName() {
+        return AES;
+    }
+
+    @Override
+    public void setCipherName(String value) {
+        ValidateUtils.checkTrue(value != null && value.equalsIgnoreCase(AES),
+                "OpenSSHKeyEncryptionContext works only with AES encryption");
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * @deprecated Use {@link #getPassphrase()} instead
+     */
+    @Deprecated
+    @Override
+    public String getPassword() {
+        return passphrase == null ? null : new String(passphrase);
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * @deprecated Use {@link #setPassphrase()} instead
+     */
+    @Deprecated
+    @Override
+    public void setPassword(String value) {
+        setPassphrase(value.toCharArray());
+    }
+
+    /**
+     * Retrieves a <em>copy</em> of the internally stored passphrase. The caller is
+     * responsible for clearing the returned array when it is no longer needed.
+     *
+     * @return the passphrase
+     */
+    public char[] getPassphrase() {
+        return passphrase == null ? null : passphrase.clone();
+    }
+
+    /**
+     * Stores a <em>copy</em> of the passphrase. The caller is responsible for
+     * eventually clearing the array passed as an argument, and for {@link #clear()
+     * clearing} this {@link OpenSSHKeyEncryptionContext} once it is no longer
+     * needed.
+     *
+     * @param passphrase to store
+     */
+    public void setPassphrase(char[] passphrase) {
+        if (this.passphrase != null) {
+            Arrays.fill(this.passphrase, '\000');
+        }
+        this.passphrase = passphrase == null ? null : passphrase.clone();
+    }
+
+    /**
+     * Retrieves the number of KDF rounds to apply.
+     *
+     * @return the default number of KDF rounds, >= {@link #DEFAULT_KDF_ROUNDS}
+     */
+    public int getKdfRounds() {
+        return kdfRounds;
+    }
+
+    /**
+     * Sets the number of KDF rounds to apply. If smaller than the
+     * {@link #DEFAULT_KDF_ROUNDS}, set that default.
+     *
+     * @param rounds number of rounds to apply
+     */
+    public void setKdfRounds(int rounds) {
+        this.kdfRounds = Math.max(DEFAULT_KDF_ROUNDS, rounds);
+    }
+
+    /**
+     * Retrieves the cipher's factory name.
+     *
+     * @return the name
+     */
+    protected String getCipherFactoryName() {
+        return getCipherName().toLowerCase() + getCipherType() + '-' + getCipherMode().toLowerCase();
+    }
+
+    /**
+     * Clears internal sensitive data (the password and the init vector).
+     */
+    public void clear() {
+        if (this.passphrase != null) {
+            Arrays.fill(this.passphrase, '\000');
+            this.passphrase = null;
+        }
+        byte[] iv = getInitVector();
+        if (iv != null) {
+            Arrays.fill(iv, (byte) 0);
+        }
+    }
+
+}
diff --git a/sshd-common/src/main/java/org/apache/sshd/common/config/keys/writer/openssh/OpenSSHKeyPairResourceWriter.java b/sshd-common/src/main/java/org/apache/sshd/common/config/keys/writer/openssh/OpenSSHKeyPairResourceWriter.java
new file mode 100644
index 0000000..bc8ae46
--- /dev/null
+++ b/sshd-common/src/main/java/org/apache/sshd/common/config/keys/writer/openssh/OpenSSHKeyPairResourceWriter.java
@@ -0,0 +1,335 @@
+/*
+ * 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.config.keys.writer.openssh;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.StandardCharsets;
+import java.security.GeneralSecurityException;
+import java.security.KeyPair;
+import java.security.PrivateKey;
+import java.security.PublicKey;
+import java.security.SecureRandom;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.sshd.common.cipher.BuiltinCiphers;
+import org.apache.sshd.common.cipher.CipherInformation;
+import org.apache.sshd.common.config.keys.KeyEntryResolver;
+import org.apache.sshd.common.config.keys.KeyUtils;
+import org.apache.sshd.common.config.keys.PrivateKeyEntryDecoder;
+import org.apache.sshd.common.config.keys.PublicKeyEntry;
+import org.apache.sshd.common.config.keys.PublicKeyEntryDecoder;
+import org.apache.sshd.common.config.keys.loader.AESPrivateKeyObfuscator;
+import org.apache.sshd.common.config.keys.loader.PrivateKeyEncryptionContext;
+import org.apache.sshd.common.config.keys.loader.openssh.OpenSSHKeyPairResourceParser;
+import org.apache.sshd.common.config.keys.loader.openssh.OpenSSHParserContext;
+import org.apache.sshd.common.config.keys.loader.openssh.kdf.BCrypt;
+import org.apache.sshd.common.config.keys.writer.KeyPairResourceWriter;
+import org.apache.sshd.common.util.ValidateUtils;
+import org.apache.sshd.common.util.io.SecureByteArrayOutputStream;
+
+/**
+ * A {@link KeyPairResourceWriter} for writing keys in the modern OpenSSH format, using
+ * the OpenBSD bcrypt KDF for passphrase-protected encrypted private keys.
+ */
+public class OpenSSHKeyPairResourceWriter implements KeyPairResourceWriter<OpenSSHKeyEncryptionContext> {
+
+    private static final Pattern VERTICALSPACE = Pattern.compile("\\v"); //$NON-NLS-1$
+
+    private static final String DASHES = "-----"; //$NON-NLS-1$
+
+    private static final int LINE_LENGTH = 70;
+
+    public OpenSSHKeyPairResourceWriter() {
+       super();
+    }
+
+    @Override
+    public void writePrivateKey(KeyPair key, String comment, OpenSSHKeyEncryptionContext options, SecureByteArrayOutputStream out)
+            throws IOException, GeneralSecurityException {
+        ValidateUtils.checkNotNull(key, "Cannot write null key");
+        String keyType = KeyUtils.getKeyType(key);
+        if (keyType == null) {
+            throw new GeneralSecurityException("Unsupported key: " + key.getClass().getName());
+        }
+        OpenSSHKeyEncryptionContext opt = determineEncryption(options);
+        // See https://github.com/openssh/openssh-portable/blob/master/PROTOCOL.key
+        write(out, DASHES + OpenSSHKeyPairResourceParser.BEGIN_MARKER + DASHES); // $NON-NLS-1$
+        // OpenSSH expects a single \n here, not a system line terminator!
+        out.write('\n');
+        String cipherName = OpenSSHParserContext.NONE_CIPHER;
+        int blockSize = 8; // OpenSSH "none" cipher has block size 8
+        if (opt != null) {
+            cipherName = opt.getCipherFactoryName();
+            CipherInformation spec = BuiltinCiphers.fromFactoryName(cipherName);
+            if (spec == null) {
+                // Internal error, no translation
+                throw new IllegalArgumentException("Unsupported cipher " + cipherName); //$NON-NLS-1$
+            }
+            blockSize = spec.getCipherBlockSize();
+        }
+        byte[] privateBytes = encodePrivateKey(key, keyType, blockSize, comment);
+        String kdfName = OpenSSHParserContext.NONE_CIPHER;
+        byte[] kdfOptions = new byte[0];
+        try (SecureByteArrayOutputStream bytes = new SecureByteArrayOutputStream()) {
+            write(bytes, OpenSSHKeyPairResourceParser.AUTH_MAGIC);
+            bytes.write(0);
+            if (opt != null) {
+                KeyEncryptor encryptor = new KeyEncryptor(opt);
+                opt.setPrivateKeyObfuscator(encryptor);
+                try {
+                    byte[] encodedBytes = encryptor.applyPrivateKeyCipher(privateBytes, opt, true);
+                    Arrays.fill(privateBytes, (byte) 0);
+                    privateBytes = encodedBytes;
+                    kdfName = "bcrypt"; //$NON-NLS-1$
+                    kdfOptions = encryptor.getKdfOptions();
+                } finally {
+                    opt.clear();
+                }
+            }
+            KeyEntryResolver.encodeString(bytes, cipherName);
+            KeyEntryResolver.encodeString(bytes, kdfName);
+            KeyEntryResolver.writeRLEBytes(bytes, kdfOptions);
+            KeyEntryResolver.encodeInt(bytes, 1); // 1 key only.
+            KeyEntryResolver.writeRLEBytes(bytes, encodePublicKey(key.getPublic(), keyType));
+            KeyEntryResolver.writeRLEBytes(bytes, privateBytes);
+            write(out, bytes.toByteArray(), LINE_LENGTH);
+        } finally {
+            Arrays.fill(privateBytes, (byte) 0);
+        }
+        write(out, DASHES + OpenSSHKeyPairResourceParser.END_MARKER + DASHES); // $NON-NLS-1$
+        out.write('\n');
+    }
+
+    private static OpenSSHKeyEncryptionContext determineEncryption(OpenSSHKeyEncryptionContext options) {
+        if (options == null) {
+            return null;
+        }
+        char[] passphrase = options.getPassphrase();
+        if (passphrase == null) {
+            return null;
+        }
+        try {
+            for (char ch : passphrase) {
+                if (!Character.isWhitespace(ch)) {
+                    return options;
+                }
+            }
+        } finally {
+            Arrays.fill(passphrase, '\000');
+        }
+        return null;
+    }
+
+    private static byte[] encodePrivateKey(KeyPair key, String keyType, int blockSize, String comment)
+            throws IOException, GeneralSecurityException {
+        try (SecureByteArrayOutputStream out = new SecureByteArrayOutputStream()) {
+            int check = new SecureRandom().nextInt();
+            KeyEntryResolver.encodeInt(out, check);
+            KeyEntryResolver.encodeInt(out, check);
+            KeyEntryResolver.encodeString(out, keyType);
+            @SuppressWarnings("unchecked") // Problem with generics
+            PrivateKeyEntryDecoder<PublicKey, PrivateKey> encoder = (PrivateKeyEntryDecoder<PublicKey, PrivateKey>) OpenSSHKeyPairResourceParser
+                    .getPrivateKeyEntryDecoder(keyType);
+            if (encoder.encodePrivateKey(out, key.getPrivate(), key.getPublic()) == null) {
+                throw new GeneralSecurityException("Cannot encode key of type " + keyType);
+            }
+            KeyEntryResolver.encodeString(out, comment == null ? "" : comment); //$NON-NLS-1$
+            if (blockSize > 1) {
+                // Padding
+                int size = out.size();
+                int extra = size % blockSize;
+                if (extra != 0) {
+                    for (int i = 1; i <= blockSize - extra; i++) {
+                        out.write(i & 0xFF);
+                    }
+                }
+            }
+            return out.toByteArray();
+        }
+    }
+
+    private static byte[] encodePublicKey(PublicKey key, String keyType) throws IOException, GeneralSecurityException {
+        @SuppressWarnings("unchecked") // Problem with generics.
+        PublicKeyEntryDecoder<PublicKey, ?> decoder = (PublicKeyEntryDecoder<PublicKey, ?>) KeyUtils
+                .getPublicKeyEntryDecoder(keyType);
+        if (decoder == null) {
+            throw new GeneralSecurityException("Unknown key type: " + keyType);
+        }
+        try (ByteArrayOutputStream out = new ByteArrayOutputStream()) {
+            decoder.encodePublicKey(out, key);
+            return out.toByteArray();
+        }
+    }
+
+    private static void write(OutputStream out, byte[] bytes, int lineLength) throws IOException {
+        byte[] encoded = Base64.getEncoder().encode(bytes);
+        Arrays.fill(bytes, (byte) 0);
+        int last = encoded.length;
+        for (int i = 0; i < last; i += lineLength) {
+            if (i + lineLength <= last) {
+                out.write(encoded, i, lineLength);
+            } else {
+                out.write(encoded, i, last - i);
+            }
+            out.write('\n');
+        }
+        Arrays.fill(encoded, (byte) 0);
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * Writes the public key in the single-line OpenSSH format "key-type pub-key
+     * comment" without terminating line ending. If the comment has multiple lines,
+     * only the first line is written.
+     */
+    @Override
+    public void writePublicKey(KeyPair key, String comment, OutputStream out)
+            throws IOException, GeneralSecurityException {
+        writePublicKey(key.getPublic(), comment, out);
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * Writes the public key in the single-line OpenSSH format "key-type pub-key
+     * comment" without terminating line ending. If the comment has multiple lines,
+     * only the first line is written.
+     */
+    @Override
+    public void writePublicKey(PublicKey key, String comment, OutputStream out)
+            throws IOException, GeneralSecurityException {
+        StringBuilder b = new StringBuilder();
+        PublicKeyEntry.appendPublicKeyEntry(b, key);
+        // Append first line of comment
+        if (comment != null) {
+            String line = firstLine(comment);
+            if (line != null && !line.isEmpty()) {
+                b.append(' ').append(line);
+            }
+        }
+        write(out, b.toString());
+    }
+
+    private static String firstLine(String text) {
+        Matcher m = VERTICALSPACE.matcher(text);
+        if (m.find()) {
+            return text.substring(0, m.start());
+        }
+        return text;
+    }
+
+    private static void write(OutputStream out, String s) throws IOException {
+        out.write(s.getBytes(StandardCharsets.UTF_8));
+    }
+
+    /**
+     * A key encryptor for modern-style OpenSSH private keys using the bcrypt KDF.
+     */
+    private static class KeyEncryptor extends AESPrivateKeyObfuscator {
+
+        private static final int BCRYPT_SALT_LENGTH = 16;
+
+        private final OpenSSHKeyEncryptionContext options;
+
+        private byte[] kdfOptions;
+
+        public KeyEncryptor(OpenSSHKeyEncryptionContext options) {
+            this.options = Objects.requireNonNull(options);
+        }
+
+        /**
+         * Retrieves the KDF options used. Valid only after
+         * {@link #deriveEncryptionKey(PrivateKeyEncryptionContext, int)} has been
+         * called.
+         *
+         * @return the number of KDF rounds applied
+         */
+        public byte[] getKdfOptions() {
+            return kdfOptions;
+        }
+
+        /**
+         * Derives an encryption key and set the IV on the {@code context} from the
+         * passphase provided by the context using the OpenBSD {@link BCrypt} KDF.
+         *
+         * @param context   for the encryption, provides the passphrase and transports
+         *                  other encryption-related information including the IV
+         * @param keyLength number of key bytes to generate
+         * @return {@code keyLength} bytes to use as encryption key
+         */
+        @Override
+        protected byte[] deriveEncryptionKey(PrivateKeyEncryptionContext context, int keyLength)
+                throws GeneralSecurityException {
+            byte[] iv = context.getInitVector();
+            if (iv == null) {
+                iv = generateInitializationVector(context);
+            }
+            byte[] kdfOutput = new byte[keyLength + iv.length];
+            byte[] salt = new byte[BCRYPT_SALT_LENGTH];
+            BCrypt bcrypt = new BCrypt();
+            SecureRandom random = new SecureRandom();
+            random.nextBytes(salt);
+            int rounds = options.getKdfRounds();
+            byte[] pwd = null;
+            byte[] result = null;
+            // "kdf" collects the salt and number of rounds; not sensitive data.
+            try (ByteArrayOutputStream kdf = new ByteArrayOutputStream()) {
+                pwd = convert(options.getPassphrase());
+                bcrypt.pbkdf(pwd, salt, rounds, kdfOutput);
+                KeyEntryResolver.writeRLEBytes(kdf, salt);
+                KeyEntryResolver.encodeInt(kdf, rounds);
+                kdfOptions = kdf.toByteArray();
+                context.setInitVector(Arrays.copyOfRange(kdfOutput, keyLength, kdfOutput.length));
+                result = Arrays.copyOf(kdfOutput, keyLength);
+            } catch (IOException impossible) {
+                // Never occurs with a ByteArrayOutputStream
+            } finally {
+                Arrays.fill(kdfOutput, (byte) 0); // Contains the IV at the end
+                if (pwd != null) {
+                    Arrays.fill(pwd, (byte) 0);
+                }
+            }
+            return result;
+        }
+
+        private byte[] convert(char[] pass) {
+            if (pass == null) {
+                return new byte[0];
+            }
+            ByteBuffer bytes = StandardCharsets.UTF_8.encode(CharBuffer.wrap(pass));
+            byte[] pwd = new byte[bytes.remaining()];
+            bytes.get(pwd);
+            if (bytes.hasArray()) {
+                Arrays.fill(bytes.array(), (byte) 0);
+            }
+            Arrays.fill(pass, '\000');
+            return pwd;
+        }
+    }
+}
diff --git a/sshd-common/src/main/java/org/apache/sshd/common/util/io/SecureByteArrayOutputStream.java b/sshd-common/src/main/java/org/apache/sshd/common/util/io/SecureByteArrayOutputStream.java
new file mode 100644
index 0000000..a1e9d44
--- /dev/null
+++ b/sshd-common/src/main/java/org/apache/sshd/common/util/io/SecureByteArrayOutputStream.java
@@ -0,0 +1,61 @@
+/*
+ * 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.util.io;
+
+import java.io.ByteArrayOutputStream;
+import java.util.Arrays;
+
+/**
+ * A {@link ByteArrayOutputStream} that clears its internal buffer after
+ * resizing and when it is {@link #close() closed}.
+ */
+public final class SecureByteArrayOutputStream extends ByteArrayOutputStream {
+
+    public SecureByteArrayOutputStream() {
+        super();
+    }
+
+    public SecureByteArrayOutputStream(int initialSize) {
+        super(initialSize);
+    }
+
+    @Override
+    public void close() {
+        Arrays.fill(buf, (byte) 0);
+    }
+
+    @Override
+    public void write(int b) {
+        byte[] oldBuf = buf;
+        super.write(b);
+        if (buf != oldBuf) {
+            Arrays.fill(oldBuf, (byte) 0);
+        }
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) {
+        byte[] oldBuf = buf;
+        super.write(b, off, len);
+        if (buf != oldBuf) {
+            Arrays.fill(oldBuf, (byte) 0);
+        }
+    }
+}
diff --git a/sshd-common/src/main/java/org/apache/sshd/common/util/security/eddsa/OpenSSHEd25519PrivateKeyEntryDecoder.java b/sshd-common/src/main/java/org/apache/sshd/common/util/security/eddsa/OpenSSHEd25519PrivateKeyEntryDecoder.java
index 9847651..e9becf4 100644
--- a/sshd-common/src/main/java/org/apache/sshd/common/util/security/eddsa/OpenSSHEd25519PrivateKeyEntryDecoder.java
+++ b/sshd-common/src/main/java/org/apache/sshd/common/util/security/eddsa/OpenSSHEd25519PrivateKeyEntryDecoder.java
@@ -21,7 +21,6 @@ package org.apache.sshd.common.util.security.eddsa;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.OutputStream;
 import java.security.GeneralSecurityException;
 import java.security.InvalidKeyException;
 import java.security.KeyFactory;
@@ -44,6 +43,7 @@ import org.apache.sshd.common.config.keys.impl.AbstractPrivateKeyEntryDecoder;
 import org.apache.sshd.common.keyprovider.KeyPairProvider;
 import org.apache.sshd.common.session.SessionContext;
 import org.apache.sshd.common.util.GenericUtils;
+import org.apache.sshd.common.util.io.SecureByteArrayOutputStream;
 import org.apache.sshd.common.util.security.SecurityUtils;
 
 /**
@@ -130,7 +130,7 @@ public class OpenSSHEd25519PrivateKeyEntryDecoder extends AbstractPrivateKeyEntr
     }
 
     @Override
-    public String encodePrivateKey(OutputStream s, EdDSAPrivateKey key) throws IOException {
+    public String encodePrivateKey(SecureByteArrayOutputStream s, EdDSAPrivateKey key, EdDSAPublicKey pubKey) throws IOException {
         Objects.requireNonNull(key, "No private key provided");
 
         // ed25519 bernstein naming: pk .. public key, sk .. secret key
diff --git a/sshd-common/src/test/java/org/apache/sshd/common/config/keys/writer/openssh/OpenSSHKeyPairResourceWriterTest.java b/sshd-common/src/test/java/org/apache/sshd/common/config/keys/writer/openssh/OpenSSHKeyPairResourceWriterTest.java
new file mode 100644
index 0000000..f52f054
--- /dev/null
+++ b/sshd-common/src/test/java/org/apache/sshd/common/config/keys/writer/openssh/OpenSSHKeyPairResourceWriterTest.java
@@ -0,0 +1,358 @@
+/*
+ * 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.config.keys.writer.openssh;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.StreamCorruptedException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ByteChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.security.GeneralSecurityException;
+import java.security.KeyPair;
+import java.security.KeyPairGenerator;
+import java.security.PublicKey;
+import java.security.spec.AlgorithmParameterSpec;
+import java.security.spec.ECGenParameterSpec;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.sshd.common.config.keys.AuthorizedKeyEntry;
+import org.apache.sshd.common.config.keys.FilePasswordProvider;
+import org.apache.sshd.common.config.keys.KeyUtils;
+import org.apache.sshd.common.config.keys.PublicKeyEntryResolver;
+import org.apache.sshd.common.util.io.SecureByteArrayOutputStream;
+import org.apache.sshd.common.util.io.resource.PathResource;
+import org.apache.sshd.common.util.security.SecurityUtils;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
+
+import net.i2p.crypto.eddsa.EdDSAKey;
+import net.i2p.crypto.eddsa.spec.EdDSAGenParameterSpec;
+
+@RunWith(Parameterized.class)
+public class OpenSSHKeyPairResourceWriterTest {
+
+    @Rule
+    public TemporaryFolder folder = new TemporaryFolder();
+
+    private static class TestData {
+        public final String algorithm;
+
+        public final String provider;
+
+        public final int keySize;
+
+        public final AlgorithmParameterSpec spec;
+
+        public TestData(String algorithm, int keySize,
+                AlgorithmParameterSpec spec) {
+            this(algorithm, null, keySize, spec);
+        }
+
+        public TestData(String algorithm, String provider, int keySize,
+                AlgorithmParameterSpec spec) {
+            this.algorithm = algorithm;
+            this.provider = provider;
+            this.keySize = keySize;
+            this.spec = spec;
+        }
+
+        @Override
+        public String toString() {
+            return algorithm + '-' + keySize
+                    + (provider == null ? "" : '(' + provider + ')');
+        }
+    }
+
+    @Parameters(name = "{0}")
+    public static Object[] getParams() {
+        List<TestData> result = new ArrayList<>();
+        result.add(new TestData("RSA", 1024, null));
+        result.add(new TestData("RSA", 2048, null));
+        result.add(new TestData("DSA", 1024, null));
+        result.add(
+                new TestData("ECDSA", 256,
+                        new ECGenParameterSpec("secp256r1")));
+        result.add(
+                new TestData("ECDSA", 384,
+                        new ECGenParameterSpec("secp384r1")));
+        result.add(
+                new TestData("ECDSA", 521,
+                        new ECGenParameterSpec("secp521r1")));
+        result.add(new TestData("EDDSA", "EdDSA", 25519,
+                new EdDSAGenParameterSpec("Ed25519")));
+        // Note: BC also has an EDDSA provider, but that one returns
+        // "Ed25519" as algorithm from its keys, while the one in
+        // net.i2p.crypto.eddsa gives keys with "EDDSA" as algorithm.
+        // sshd handles only the latter.
+        return result.toArray();
+    }
+
+    @Parameter
+    public TestData data;
+
+    private KeyPair testKey;
+
+    private OpenSSHKeyPairResourceWriter writer = new OpenSSHKeyPairResourceWriter();
+
+    @Before
+    public void generateKey() throws Exception {
+        KeyPairGenerator generator;
+        if (data.provider == null) {
+            generator = KeyPairGenerator.getInstance(data.algorithm);
+        } else {
+            generator = KeyPairGenerator.getInstance(data.algorithm,
+                    data.provider);
+        }
+        if (data.spec != null) {
+            generator.initialize(data.spec);
+        } else {
+            generator.initialize(data.keySize);
+        }
+        testKey = generator.generateKeyPair();
+    }
+
+    private boolean compare(KeyPair a, KeyPair b) {
+        if ("EDDSA".equals(data.algorithm)) {
+            // Bug in net.i2p.crypto.eddsa and in sshd? Both also compare the
+            // seed of the private key, but for a generated key, this is some
+            // random value, while it is all zeroes for a key read from a file.
+            return KeyUtils.compareKeys(a.getPublic(), b.getPublic())
+                    && Objects.equals(((EdDSAKey) a.getPrivate()).getParams(),
+                            ((EdDSAKey) b.getPrivate()).getParams());
+        }
+        // Compares both public and private keys.
+        return KeyUtils.compareKeyPairs(a, b);
+    }
+
+    private static void writeToFile(File file, byte[] sensitiveData)
+            throws IOException {
+        try (ByteChannel out = Files.newByteChannel(file.toPath(),
+                StandardOpenOption.CREATE, StandardOpenOption.WRITE)) {
+            ByteBuffer buf = ByteBuffer.wrap(sensitiveData);
+            while (buf.hasRemaining()) {
+                out.write(buf);
+            }
+        } finally {
+            Arrays.fill(sensitiveData, (byte) 0);
+        }
+    }
+
+    @Test
+    public void privateNoEncryption() throws Exception {
+        File tmp = folder.newFile();
+        try (SecureByteArrayOutputStream out = new SecureByteArrayOutputStream()) {
+            writer.writePrivateKey(testKey, "a comment", null, out);
+            writeToFile(tmp, out.toByteArray());
+        }
+        try (InputStream in = Files.newInputStream(tmp.toPath())) {
+            KeyPair key = SecurityUtils.loadKeyPairIdentities(null,
+                    new PathResource(tmp.toPath()), in, null).iterator().next();
+            assertNotNull(key);
+            assertTrue("Keys should be equal", compare(key, testKey));
+        }
+    }
+
+    @Test
+    public void privateNoPassword() throws Exception {
+        File tmp = folder.newFile();
+        OpenSSHKeyEncryptionContext options = new OpenSSHKeyEncryptionContext();
+        try (SecureByteArrayOutputStream out = new SecureByteArrayOutputStream()) {
+            writer.writePrivateKey(testKey, "a comment", options, out);
+            writeToFile(tmp, out.toByteArray());
+        } finally {
+            options.clear();
+        }
+        try (InputStream in = Files.newInputStream(tmp.toPath())) {
+            KeyPair key = SecurityUtils.loadKeyPairIdentities(null,
+                    new PathResource(tmp.toPath()), in, null).iterator().next();
+            assertNotNull(key);
+            assertTrue("Keys should be equal", compare(key, testKey));
+        }
+    }
+
+    @Test
+    public void privateEncryptedAesCbc128() throws Exception {
+        File tmp = folder.newFile();
+        OpenSSHKeyEncryptionContext options = new OpenSSHKeyEncryptionContext();
+        try (SecureByteArrayOutputStream out = new SecureByteArrayOutputStream()) {
+            options.setPassphrase("nonsense".toCharArray());
+            options.setCipherName("AES");
+            options.setCipherMode("CBC");
+            options.setCipherType("128");
+            writer.writePrivateKey(testKey, "a comment", options, out);
+            writeToFile(tmp, out.toByteArray());
+        } finally {
+            options.clear();
+        }
+        try (InputStream in = Files.newInputStream(tmp.toPath())) {
+            KeyPair key = SecurityUtils
+                    .loadKeyPairIdentities(null, new PathResource(tmp.toPath()),
+                            in, FilePasswordProvider.of("nonsense"))
+                    .iterator().next();
+            assertNotNull(key);
+            assertTrue("Keys should be equal", compare(key, testKey));
+        }
+    }
+
+    @Test
+    public void privateEncryptedAesCtr256() throws Exception {
+        File tmp = folder.newFile();
+        OpenSSHKeyEncryptionContext options = new OpenSSHKeyEncryptionContext();
+        try (SecureByteArrayOutputStream out = new SecureByteArrayOutputStream()) {
+            options.setPassphrase("nonsense".toCharArray());
+            options.setCipherName("AES");
+            options.setCipherMode("CTR");
+            options.setCipherType("256");
+            writer.writePrivateKey(testKey, "a comment", options, out);
+            writeToFile(tmp, out.toByteArray());
+        } finally {
+            options.clear();
+        }
+        try (InputStream in = Files.newInputStream(tmp.toPath())) {
+            KeyPair key = SecurityUtils
+                    .loadKeyPairIdentities(null, new PathResource(tmp.toPath()),
+                            in, FilePasswordProvider.of("nonsense"))
+                    .iterator().next();
+            assertNotNull(key);
+            assertTrue("Keys should be equal", compare(key, testKey));
+        }
+    }
+
+    @Test
+    public void privateEncryptedWrongPassword() throws Exception {
+        File tmp = folder.newFile();
+        OpenSSHKeyEncryptionContext options = new OpenSSHKeyEncryptionContext();
+        try (SecureByteArrayOutputStream out = new SecureByteArrayOutputStream()) {
+            options.setPassphrase("nonsense".toCharArray());
+            options.setCipherName("AES");
+            options.setCipherMode("CTR");
+            options.setCipherType("256");
+            writer.writePrivateKey(testKey, "a comment", options, out);
+            writeToFile(tmp, out.toByteArray());
+        } finally {
+            options.clear();
+        }
+        try (InputStream in = Files.newInputStream(tmp.toPath())) {
+            SecurityUtils.loadKeyPairIdentities(null,
+                    new PathResource(tmp.toPath()), in,
+                    FilePasswordProvider.of("wrong"));
+            fail("Expected an exception");
+        } catch (StreamCorruptedException | GeneralSecurityException e) {
+            // Expected
+        }
+    }
+
+    @Test
+    public void privateEncryptedNoPassword() throws Exception {
+        File tmp = folder.newFile();
+        OpenSSHKeyEncryptionContext options = new OpenSSHKeyEncryptionContext();
+        try (SecureByteArrayOutputStream out = new SecureByteArrayOutputStream()) {
+            options.setPassphrase("nonsense".toCharArray());
+            options.setCipherName("AES");
+            options.setCipherMode("CTR");
+            options.setCipherType("256");
+            writer.writePrivateKey(testKey, "a comment", options, out);
+            writeToFile(tmp, out.toByteArray());
+        } finally {
+            options.clear();
+        }
+        try (InputStream in = Files.newInputStream(tmp.toPath())) {
+            assertThrows(GeneralSecurityException.class,
+                    () -> SecurityUtils.loadKeyPairIdentities(null,
+                            new PathResource(tmp.toPath()), in, null));
+        }
+    }
+
+    private void checkPublicKey(File tmp, String comment) throws Exception {
+        List<AuthorizedKeyEntry> keysRead = AuthorizedKeyEntry
+                .readAuthorizedKeys(tmp.toPath());
+        assertEquals("Unexpected list size", 1, keysRead.size());
+        AuthorizedKeyEntry entry = keysRead.get(0);
+        String readComment = entry.getComment();
+        if (comment == null || comment.isEmpty()) {
+            assertTrue("Unexpected comment", readComment == null || readComment.isEmpty());
+        } else {
+            assertEquals("Unexpected comment", comment, readComment);
+        }
+        PublicKey pubKey = entry.resolvePublicKey(null,
+                PublicKeyEntryResolver.IGNORING);
+        assertTrue("keys don't match",
+                KeyUtils.compareKeys(testKey.getPublic(), pubKey));
+    }
+
+    @Test
+    public void publicWithComment() throws Exception {
+        File tmp = folder.newFile();
+        try (OutputStream out = new FileOutputStream(tmp)) {
+            writer.writePublicKey(testKey, "a comment", out);
+        }
+        checkPublicKey(tmp, "a comment");
+    }
+
+    @Test
+    public void publicWithMultilineComment() throws Exception {
+        File tmp = folder.newFile();
+        try (OutputStream out = new FileOutputStream(tmp)) {
+            writer.writePublicKey(testKey,
+                    "a comment" + System.lineSeparator() + "second line", out);
+        }
+        assertEquals("Unexpected number of lines", 1,
+                Files.readAllLines(tmp.toPath()).size());
+        checkPublicKey(tmp, "a comment");
+    }
+
+    @Test
+    public void publicNoComment() throws Exception {
+        File tmp = folder.newFile();
+        try (OutputStream out = new FileOutputStream(tmp)) {
+            writer.writePublicKey(testKey, null, out);
+        }
+        checkPublicKey(tmp, null);
+    }
+
+    @Test
+    public void publicEmptyComment() throws Exception {
+        File tmp = folder.newFile();
+        try (OutputStream out = new FileOutputStream(tmp)) {
+            writer.writePublicKey(testKey, "", out);
+        }
+        checkPublicKey(tmp, null);
+    }
+}