You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@karaf.apache.org by cs...@apache.org on 2017/08/02 09:24:18 UTC

karaf git commit: Use plain sshd FileKeyPairProvider for reading

Repository: karaf
Updated Branches:
  refs/heads/OPENSSH [created] 1e4335922


Use plain sshd FileKeyPairProvider for reading


Project: http://git-wip-us.apache.org/repos/asf/karaf/repo
Commit: http://git-wip-us.apache.org/repos/asf/karaf/commit/1e433592
Tree: http://git-wip-us.apache.org/repos/asf/karaf/tree/1e433592
Diff: http://git-wip-us.apache.org/repos/asf/karaf/diff/1e433592

Branch: refs/heads/OPENSSH
Commit: 1e4335922d232bfc8591618e5c935d98891d5bf2
Parents: f7ea3e0
Author: Christian Schneider <ch...@die-schneider.net>
Authored: Wed Aug 2 11:24:05 2017 +0200
Committer: Christian Schneider <ch...@die-schneider.net>
Committed: Wed Aug 2 11:24:05 2017 +0200

----------------------------------------------------------------------
 .../org/apache/karaf/shell/ssh/Activator.java   | 41 +++++-----
 .../ssh/OpenSSHGeneratorFileKeyProvider.java    | 82 --------------------
 .../keygenerator/OpenSSHKeyPairGenerator.java   | 54 +++++++++++++
 .../karaf/shell/ssh/keygenerator/PemWriter.java | 50 ++++++++++++
 .../OpenSSHGeneratorKeyFileProviderTest.java    | 62 ---------------
 .../OpenSSHGeneratorKeyFileProviderTest.java    | 47 +++++++++++
 6 files changed, 170 insertions(+), 166 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/karaf/blob/1e433592/shell/ssh/src/main/java/org/apache/karaf/shell/ssh/Activator.java
----------------------------------------------------------------------
diff --git a/shell/ssh/src/main/java/org/apache/karaf/shell/ssh/Activator.java b/shell/ssh/src/main/java/org/apache/karaf/shell/ssh/Activator.java
index 8725079..939127d 100644
--- a/shell/ssh/src/main/java/org/apache/karaf/shell/ssh/Activator.java
+++ b/shell/ssh/src/main/java/org/apache/karaf/shell/ssh/Activator.java
@@ -20,23 +20,26 @@ package org.apache.karaf.shell.ssh;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.Path;
 import java.nio.file.Paths;
+import java.security.KeyPair;
 import java.util.Collections;
 
 import org.apache.karaf.shell.api.action.lifecycle.Manager;
 import org.apache.karaf.shell.api.console.CommandLoggingFilter;
 import org.apache.karaf.shell.api.console.Session;
 import org.apache.karaf.shell.api.console.SessionFactory;
+import org.apache.karaf.shell.ssh.keygenerator.OpenSSHKeyPairGenerator;
+import org.apache.karaf.shell.ssh.keygenerator.PemWriter;
 import org.apache.karaf.shell.support.RegexCommandLoggingFilter;
 import org.apache.karaf.util.tracker.BaseActivator;
 import org.apache.karaf.util.tracker.annotation.Managed;
 import org.apache.karaf.util.tracker.annotation.RequireService;
 import org.apache.karaf.util.tracker.annotation.Services;
 import org.apache.sshd.common.file.virtualfs.VirtualFileSystemFactory;
+import org.apache.sshd.common.keyprovider.FileKeyPairProvider;
 import org.apache.sshd.server.SshServer;
 import org.apache.sshd.server.forward.AcceptAllForwardingFilter;
-import org.apache.sshd.server.keyprovider.AbstractGeneratorHostKeyProvider;
-import org.apache.sshd.server.keyprovider.SimpleGeneratorHostKeyProvider;
 import org.apache.sshd.server.scp.ScpCommandFactory;
 import org.apache.sshd.server.subsystem.sftp.SftpSubsystemFactory;
 import org.osgi.framework.ServiceReference;
@@ -143,7 +146,6 @@ public class Activator extends BaseActivator implements ManagedService {
         int nioWorkers         = getInt("nio-workers", 2);
         String sshRealm        = getString("sshRealm", "karaf");
         String hostKey         = getString("hostKey", System.getProperty("karaf.etc") + "/host.key");
-        String hostKeyFormat   = getString("hostKeyFormat", "simple");
         String[] authMethods   = getStringArray("authMethods", "keyboard-interactive,password,publickey");
         int keySize            = getInt("keySize", 4096);
         String algorithm       = getString("algorithm", "RSA");
@@ -152,28 +154,14 @@ public class Activator extends BaseActivator implements ManagedService {
         String[] kexAlgorithms = getStringArray("kexAlgorithms", "diffie-hellman-group-exchange-sha256,ecdh-sha2-nistp521,ecdh-sha2-nistp384,ecdh-sha2-nistp256,diffie-hellman-group-exchange-sha1,diffie-hellman-group1-sha1");
         String welcomeBanner   = getString("welcomeBanner", null);
         String moduliUrl       = getString("moduli-url", null);
-
-        AbstractGeneratorHostKeyProvider keyPairProvider;
-        if ("simple".equalsIgnoreCase(hostKeyFormat)) {
-            keyPairProvider = new SimpleGeneratorHostKeyProvider();
-        } else if ("PEM".equalsIgnoreCase(hostKeyFormat)) {
-            keyPairProvider = new OpenSSHGeneratorFileKeyProvider();
-        } else {
-            LOGGER.error("Invalid host key format " + hostKeyFormat);
-            return null;
-        }
-
-        keyPairProvider.setPath(Paths.get(hostKey));
-        if (new File(hostKey).exists()) {
-            // do not trash key file if there's something wrong with it.
-            keyPairProvider.setOverwriteAllowed(false);
-        } else {
-            keyPairProvider.setKeySize(keySize);
-            keyPairProvider.setAlgorithm(algorithm);
+        
+        Path serverKeyPath = Paths.get(hostKey);
+        if (!serverKeyPath.toFile().exists()) {
+            createServerKey(serverKeyPath.toFile(), algorithm, keySize);
         }
 
+        FileKeyPairProvider keyPairProvider = new FileKeyPairProvider(serverKeyPath);
         KarafJaasAuthenticator authenticator = new KarafJaasAuthenticator(sshRealm);
-
         UserAuthFactoriesFactory authFactoriesFactory = new UserAuthFactoriesFactory();
         authFactoriesFactory.setAuthMethods(authMethods);
 
@@ -204,4 +192,13 @@ public class Activator extends BaseActivator implements ManagedService {
         return server;
     }
 
+    private void createServerKey(File keyFile, String algorithm, int keySize) {
+        try {
+            KeyPair kp = new OpenSSHKeyPairGenerator(algorithm, keySize).generate();
+            new PemWriter(keyFile).writeKeyPair(kp);
+        } catch (Exception e) {
+            throw new RuntimeException("Key file generation failed", e);
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/karaf/blob/1e433592/shell/ssh/src/main/java/org/apache/karaf/shell/ssh/OpenSSHGeneratorFileKeyProvider.java
----------------------------------------------------------------------
diff --git a/shell/ssh/src/main/java/org/apache/karaf/shell/ssh/OpenSSHGeneratorFileKeyProvider.java b/shell/ssh/src/main/java/org/apache/karaf/shell/ssh/OpenSSHGeneratorFileKeyProvider.java
deleted file mode 100644
index dd17314..0000000
--- a/shell/ssh/src/main/java/org/apache/karaf/shell/ssh/OpenSSHGeneratorFileKeyProvider.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.karaf.shell.ssh;
-
-import org.apache.commons.ssl.PEMItem;
-import org.apache.commons.ssl.PEMUtil;
-import org.apache.commons.ssl.PKCS8Key;
-import org.apache.sshd.server.keyprovider.AbstractGeneratorHostKeyProvider;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.file.Paths;
-import java.security.GeneralSecurityException;
-import java.security.KeyPair;
-import java.util.ArrayList;
-import java.util.Collection;
-
-/**
- * Read SSH keys without resorting to BouncyCastle
- */
-public class OpenSSHGeneratorFileKeyProvider extends AbstractGeneratorHostKeyProvider {
-    private String password;
-
-    public OpenSSHGeneratorFileKeyProvider() {
-    }
-
-    public OpenSSHGeneratorFileKeyProvider(String path) {
-        setPath(Paths.get(path));
-    }
-
-    public OpenSSHGeneratorFileKeyProvider(String path, String algorithm) {
-        this(path);
-        setAlgorithm(algorithm);
-    }
-
-    public OpenSSHGeneratorFileKeyProvider(String path, String algorithm, int keySize) {
-        this(path, algorithm);
-        setKeySize(keySize);
-    }
-
-    public String getPassword() {
-        return password;
-    }
-
-    public void setPassword(String password) {
-        this.password = password;
-    }
-
-    @Override
-    protected KeyPair doReadKeyPair(String resourceKey, InputStream is) throws IOException, GeneralSecurityException {
-        PKCS8Key pkcs8 = new PKCS8Key(is, password == null ? null : password.toCharArray());
-        return new KeyPair(pkcs8.getPublicKey(), pkcs8.getPrivateKey());
-    }
-
-    @Override
-    protected void doWriteKeyPair(String resourceKey, KeyPair kp, OutputStream os) throws IOException, GeneralSecurityException {
-        Collection<Object> items = new ArrayList<>();
-        items.add(new PEMItem(kp.getPrivate().getEncoded(), "PRIVATE KEY"));
-        byte[] bytes = PEMUtil.encode(items);
-        os.write(bytes);
-        os.close();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/karaf/blob/1e433592/shell/ssh/src/main/java/org/apache/karaf/shell/ssh/keygenerator/OpenSSHKeyPairGenerator.java
----------------------------------------------------------------------
diff --git a/shell/ssh/src/main/java/org/apache/karaf/shell/ssh/keygenerator/OpenSSHKeyPairGenerator.java b/shell/ssh/src/main/java/org/apache/karaf/shell/ssh/keygenerator/OpenSSHKeyPairGenerator.java
new file mode 100644
index 0000000..38db81e
--- /dev/null
+++ b/shell/ssh/src/main/java/org/apache/karaf/shell/ssh/keygenerator/OpenSSHKeyPairGenerator.java
@@ -0,0 +1,54 @@
+/*
+ * 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.karaf.shell.ssh.keygenerator;
+
+import java.security.GeneralSecurityException;
+import java.security.KeyPair;
+import java.security.KeyPairGenerator;
+
+import org.apache.sshd.common.cipher.ECCurves;
+import org.apache.sshd.common.config.keys.KeyUtils;
+import org.apache.sshd.common.util.security.SecurityUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OpenSSHKeyPairGenerator {
+    private Logger log = LoggerFactory.getLogger(this.getClass());
+    private int keySize;
+    private String algorithm;
+    
+    public OpenSSHKeyPairGenerator(String algorithm, int keySize) {
+        this.algorithm = algorithm;
+        this.keySize = keySize;
+    }
+
+    public KeyPair generate() throws GeneralSecurityException {
+        KeyPairGenerator generator = SecurityUtils.getKeyPairGenerator(algorithm);
+        if (keySize != 0) {
+            generator.initialize(keySize);
+            log.info("generateKeyPair(" + algorithm + ") generating host key - size=" + keySize);
+        } else if (KeyUtils.EC_ALGORITHM.equals(algorithm)) {
+            // If left to our own devices choose the biggest key size possible
+            int numCurves = ECCurves.SORTED_KEY_SIZE.size();
+            ECCurves curve = ECCurves.SORTED_KEY_SIZE.get(numCurves - 1);
+            generator.initialize(curve.getParameters());
+        }
+        return generator.generateKeyPair();
+    }
+}

http://git-wip-us.apache.org/repos/asf/karaf/blob/1e433592/shell/ssh/src/main/java/org/apache/karaf/shell/ssh/keygenerator/PemWriter.java
----------------------------------------------------------------------
diff --git a/shell/ssh/src/main/java/org/apache/karaf/shell/ssh/keygenerator/PemWriter.java b/shell/ssh/src/main/java/org/apache/karaf/shell/ssh/keygenerator/PemWriter.java
new file mode 100644
index 0000000..f5409e2
--- /dev/null
+++ b/shell/ssh/src/main/java/org/apache/karaf/shell/ssh/keygenerator/PemWriter.java
@@ -0,0 +1,50 @@
+/*
+ * 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.karaf.shell.ssh.keygenerator;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.security.KeyPair;
+import java.util.ArrayList;
+import java.util.Collection;
+
+import org.apache.commons.ssl.PEMItem;
+import org.apache.commons.ssl.PEMUtil;
+
+public class PemWriter {
+    private File keyFile;
+
+    public PemWriter(File keyFile) {
+        this.keyFile = keyFile;
+    }
+    
+    public void writeKeyPair(KeyPair kp) throws IOException, FileNotFoundException {
+        Collection<Object> items = new ArrayList<>();
+        String privAlg = kp.getPrivate().getAlgorithm();
+        items.add(new PEMItem(kp.getPrivate().getEncoded(), privAlg + " PRIVATE KEY"));
+        String pubAlg = kp.getPublic().getAlgorithm();
+        items.add(new PEMItem(kp.getPublic().getEncoded(), pubAlg + " PUBLIC KEY"));
+        byte[] bytes = PEMUtil.encode(items);
+        FileOutputStream os = new FileOutputStream(keyFile);
+        os.write(bytes);
+        os.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/karaf/blob/1e433592/shell/ssh/src/test/java/org/apache/karaf/shell/ssh/OpenSSHGeneratorKeyFileProviderTest.java
----------------------------------------------------------------------
diff --git a/shell/ssh/src/test/java/org/apache/karaf/shell/ssh/OpenSSHGeneratorKeyFileProviderTest.java b/shell/ssh/src/test/java/org/apache/karaf/shell/ssh/OpenSSHGeneratorKeyFileProviderTest.java
deleted file mode 100644
index 04066c4..0000000
--- a/shell/ssh/src/test/java/org/apache/karaf/shell/ssh/OpenSSHGeneratorKeyFileProviderTest.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.karaf.shell.ssh;
-
-import java.io.File;
-import java.math.BigInteger;
-import java.security.KeyPair;
-import java.security.interfaces.RSAPrivateCrtKey;
-import java.security.interfaces.RSAPublicKey;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-public class OpenSSHGeneratorKeyFileProviderTest {
-
-    @Test
-    public void readSshKey() throws Exception {
-        OpenSSHGeneratorFileKeyProvider prov = new OpenSSHGeneratorFileKeyProvider("src/test/resources/org/apache/karaf/shell/ssh/test.pem");
-        prov.setOverwriteAllowed(false);
-        KeyPair keys = prov.loadKeys().iterator().next();
-        // how would we tell if they read 'correctly'? Well, the base class will throw if the key isn't reasonable.
-        Assert.assertNotNull(keys);
-        Assert.assertTrue("Loaded key is not RSA Key", keys.getPublic() instanceof RSAPublicKey);
-        Assert.assertEquals(65537, ((RSAPublicKey) keys.getPublic()).getPublicExponent().intValue());
-    }
-    
-    @Test
-    public void writeSshKey() throws Exception {
-    	// create a temporary file
-    	File temp = File.createTempFile(this.getClass().getCanonicalName(), ".pem");
-    	temp.deleteOnExit();
-    	OpenSSHGeneratorFileKeyProvider prov = new OpenSSHGeneratorFileKeyProvider(temp.getPath(), "RSA", 4096);
-    	KeyPair keys = prov.loadKeys().iterator().next();
-    	Assert.assertNotNull(keys);
-    	Assert.assertTrue(temp.exists());
-    	Assert.assertFalse(temp.length() == 0);
-    	BigInteger privateExponent = ((RSAPrivateCrtKey) keys.getPrivate()).getPrivateExponent();
-    	// read and check if correctly read
-    	prov = new OpenSSHGeneratorFileKeyProvider(temp.getPath());
-    	keys = prov.loadKeys().iterator().next();
-        Assert.assertNotNull(keys);
-        Assert.assertTrue("Loaded key is not RSA Key", keys.getPrivate() instanceof RSAPrivateCrtKey);
-        BigInteger privateExponent2 = ((RSAPrivateCrtKey) keys.getPrivate()).getPrivateExponent();
-        Assert.assertEquals(privateExponent, privateExponent2);
-    }
-}

http://git-wip-us.apache.org/repos/asf/karaf/blob/1e433592/shell/ssh/src/test/java/org/apache/karaf/shell/ssh/keygenerator/OpenSSHGeneratorKeyFileProviderTest.java
----------------------------------------------------------------------
diff --git a/shell/ssh/src/test/java/org/apache/karaf/shell/ssh/keygenerator/OpenSSHGeneratorKeyFileProviderTest.java b/shell/ssh/src/test/java/org/apache/karaf/shell/ssh/keygenerator/OpenSSHGeneratorKeyFileProviderTest.java
new file mode 100644
index 0000000..fc97ab6
--- /dev/null
+++ b/shell/ssh/src/test/java/org/apache/karaf/shell/ssh/keygenerator/OpenSSHGeneratorKeyFileProviderTest.java
@@ -0,0 +1,47 @@
+/*
+ * 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.karaf.shell.ssh.keygenerator;
+
+import java.io.File;
+import java.nio.file.Path;
+import java.security.KeyPair;
+import java.security.interfaces.RSAPrivateCrtKey;
+
+import org.apache.sshd.common.config.keys.KeyUtils;
+import org.apache.sshd.common.keyprovider.FileKeyPairProvider;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class OpenSSHGeneratorKeyFileProviderTest {
+
+    @Test
+    public void writeSshKey() throws Exception {
+        File temp = File.createTempFile(this.getClass().getCanonicalName(), ".pem");
+        temp.deleteOnExit();
+
+        KeyPair kp = new OpenSSHKeyPairGenerator(KeyUtils.RSA_ALGORITHM, 1024).generate();
+        new PemWriter(temp).writeKeyPair(kp);
+
+        Path path = new File("/home/cschneider/.ssh/id_rsa").toPath();
+        FileKeyPairProvider prov = new FileKeyPairProvider(temp.toPath());
+        KeyPair keys = prov.loadKeys().iterator().next();
+        Assert.assertNotNull(keys);
+        Assert.assertTrue("Loaded key is not RSA Key", keys.getPrivate() instanceof RSAPrivateCrtKey);
+    }
+}