You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ag...@apache.org on 2020/02/21 12:03:35 UTC

[drill] 06/10: DRILL-7573: Support htpasswd based authentication

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

agozhiy pushed a commit to branch MERGE-200221-00
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 39457bf8008e73cca8c9de8a677d5f29717b6d31
Author: Dobes Vandermeer <do...@gmail.com>
AuthorDate: Mon Feb 10 21:51:54 2020 -0800

    DRILL-7573: Support htpasswd based authentication
    
    In containerized environments, PAM based authentication is not convenient.
    
    This provides a simple mechanism for setting up users' passwords
    that can be managed using docker volume mounts.
    
    closes #1977
---
 .../java/org/apache/drill/exec/ExecConstants.java  |   1 +
 .../security/HtpasswdFileUserAuthenticator.java    | 152 +++++++++++++++++++++
 .../TestHtpasswdFileUserAuthenticator.java         | 151 ++++++++++++++++++++
 3 files changed, 304 insertions(+)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index adb3a4d..5938ca1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -245,6 +245,7 @@ public final class ExecConstants {
   public static final String AUTHENTICATION_MECHANISMS = "drill.exec.security.auth.mechanisms";
   public static final String USER_AUTHENTICATION_ENABLED = "drill.exec.security.user.auth.enabled";
   public static final String USER_AUTHENTICATOR_IMPL = "drill.exec.security.user.auth.impl";
+  public static final String HTPASSWD_AUTHENTICATOR_PATH = "drill.exec.security.user.auth.htpasswd.path";
   public static final String PAM_AUTHENTICATOR_PROFILES = "drill.exec.security.user.auth.pam_profiles";
   public static final String BIT_AUTHENTICATION_ENABLED = "drill.exec.security.bit.auth.enabled";
   public static final String BIT_AUTHENTICATION_MECHANISM = "drill.exec.security.bit.auth.mechanism";
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/HtpasswdFileUserAuthenticator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/HtpasswdFileUserAuthenticator.java
new file mode 100644
index 0000000..635e8a6
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/HtpasswdFileUserAuthenticator.java
@@ -0,0 +1,152 @@
+/*
+ * 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.drill.exec.rpc.user.security;
+
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.codec.digest.Md5Crypt;
+import org.apache.commons.io.Charsets;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.text.MessageFormat;
+import java.util.Base64;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Implementation of UserAuthenticator that reads passwords from an htpasswd
+ * formatted file.
+ * <p>
+ * Currently supports MD5, SHA-1, and plaintext passwords.
+ * <p>
+ * Use the htpasswd command line tool to create and modify htpasswd files.
+ * <p>
+ * By default this loads the passwords from <code>/opt/drill/conf/htpasswd</code>.  Users can change the path by
+ * putting the absolute file path as <code>drill.exec.security.user.auth.htpasswd.path</code> in
+ * <code>drill-override.conf</code>.
+ * <p>
+ * This is intended for situations where the list of users is relatively static, and you are running
+ * drill in a container so using pam is not convenient.
+ */
+@UserAuthenticatorTemplate(type = "htpasswd")
+public class HtpasswdFileUserAuthenticator implements UserAuthenticator {
+  private static final Logger logger = LoggerFactory.getLogger(HtpasswdFileUserAuthenticator.class);
+  private static final Pattern HTPASSWD_LINE_PATTERN = Pattern.compile("^([^:]+):([^:]+)");
+  public static final String DEFAULT_HTPASSWD_AUTHENTICATOR_PATH = "/opt/drill/conf/htpasswd";
+
+  private String path = DEFAULT_HTPASSWD_AUTHENTICATOR_PATH;
+  private long lastModified;
+  private long lastFileSize;
+  private Map<String, String> userToPassword;
+
+  @Override
+  public void setup(DrillConfig drillConfig) throws DrillbitStartupException {
+    if (drillConfig.hasPath(ExecConstants.HTPASSWD_AUTHENTICATOR_PATH)) {
+      path = drillConfig.getString(ExecConstants.HTPASSWD_AUTHENTICATOR_PATH);
+    }
+  }
+
+  /**
+   * Check password against hash read from the file
+   *
+   * @param password User provided password
+   * @param hash     Hash stored in the htpasswd file
+   * @return true if the password matched the hash
+   */
+  public static boolean isPasswordValid(String password, String hash) {
+    if (hash.startsWith("$apr1$")) {
+      return hash.equals(Md5Crypt.apr1Crypt(password, hash));
+    } else if (hash.startsWith("$1$")) {
+      return hash.equals(Md5Crypt.md5Crypt(password.getBytes(Charsets.UTF_8), hash));
+    } else if (hash.startsWith("{SHA}")) {
+      return hash.substring(5).equals(Base64.getEncoder().encodeToString(DigestUtils.sha1(password)));
+    } else if (hash.startsWith("$2y$")) {
+      // bcrypt not supported currently
+      return false;
+    } else {
+      return hash.equals(password);
+    }
+  }
+
+  /**
+   * Validate the given username and password against the password file
+   *
+   * @param username Username provided
+   * @param password Password provided
+   * @throws UserAuthenticationException If the username and password could not be validated
+   */
+  @Override
+  public void authenticate(String username, String password) throws UserAuthenticationException {
+    read();
+    String hash = this.userToPassword.get(username);
+    boolean credentialsAccepted = (hash != null && isPasswordValid(password, hash));
+    if (!credentialsAccepted) {
+      throw new UserAuthenticationException(String.format("htpasswd auth failed for user '%s'",
+        username));
+    }
+  }
+
+  /**
+   * Read the password file into the map, if the file has changed since we last read it
+   */
+  protected synchronized void read() {
+    File file = new File(path);
+    long newLastModified = file.exists() ? file.lastModified() : 0;
+    long newFileSize = file.exists() ? file.length() : 0;
+    if (userToPassword == null || newLastModified != lastModified || newFileSize != lastFileSize) {
+      HashMap<String, String> newMap = new HashMap<>();
+      if(newFileSize != 0) {
+        try (BufferedReader reader = new BufferedReader(new FileReader(file))) {
+          String line;
+          while ((line = reader.readLine()) != null) {
+            if (!line.isEmpty() && !line.startsWith("#")) {
+              Matcher m = HTPASSWD_LINE_PATTERN.matcher(line);
+              if (m.matches()) {
+                newMap.put(m.group(1), m.group(2));
+              }
+            }
+          }
+        } catch (Exception e) {
+          logger.error(MessageFormat.format("Failed to read htpasswd file at path {0}", file), e);
+        }
+      } else {
+        logger.error(MessageFormat.format("Empty or missing htpasswd file at path {0}", file));
+      }
+      lastFileSize = newFileSize;
+      lastModified = newLastModified;
+      userToPassword = newMap;
+    }
+  }
+
+  /**
+   * Free resources associated with this authenticator
+   */
+  @Override
+  public void close() {
+    lastModified = 0;
+    userToPassword = null;
+  }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestHtpasswdFileUserAuthenticator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestHtpasswdFileUserAuthenticator.java
new file mode 100644
index 0000000..9c27408
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestHtpasswdFileUserAuthenticator.java
@@ -0,0 +1,151 @@
+/*
+ * 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.drill.exec.rpc.user.security;
+
+import org.apache.drill.common.config.DrillProperties;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.test.ClientFixture;
+import org.apache.drill.test.ClusterFixture;
+import org.apache.drill.test.ClusterTest;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestHtpasswdFileUserAuthenticator extends ClusterTest {
+  private File tempPasswdFile;
+
+
+  private void setupCluster(String passwdContent) throws IOException {
+    tempPasswdFile = new File(dirTestWatcher.getTmpDir(), "htpasswd." + System.currentTimeMillis());
+    Files.write(tempPasswdFile.toPath(), passwdContent.getBytes());
+
+    cluster = ClusterFixture.bareBuilder(dirTestWatcher)
+      .clusterSize(3)
+      .configProperty(ExecConstants.ALLOW_LOOPBACK_ADDRESS_BINDING, true)
+      .configProperty(ExecConstants.USER_AUTHENTICATION_ENABLED, true)
+      .configProperty(ExecConstants.USER_AUTHENTICATOR_IMPL, "htpasswd")
+      .configProperty(ExecConstants.HTPASSWD_AUTHENTICATOR_PATH, tempPasswdFile.toString())
+      .build();
+  }
+
+
+  @Test
+  public void passwordChecksGiveCorrectResults() throws Exception {
+    String passwdContent = "alice:pass1\n" +
+      "bob:buzzkill\n" +
+      "jane:$apr1$PrwDfXy9$ajkhotQW6RFnoVQtPKoW4/\n" +
+      "john:$apr1$UxZgBU8k$K4UzdubNa741TnWAZY2QV0\n";
+    setupCluster(passwdContent);
+
+
+    assertTrue(true);
+
+    tryCredentials("alice", "pass1", cluster, true);
+    tryCredentials("bob", "buzzkill", cluster, true);
+    tryCredentials("notalice", "pass1", cluster, false);
+    tryCredentials("notbob", "buzzkill", cluster, false);
+    tryCredentials("alice", "wrong", cluster, false);
+    tryCredentials("bob", "incorrect", cluster, false);
+    tryCredentials("jane", "pass", cluster, true);
+    tryCredentials("john", "foobar", cluster, true);
+    tryCredentials("jane", "wrong", cluster, false);
+    tryCredentials("john", "incorrect1", cluster, false);
+  }
+
+  @Test
+  public void rejectsLoginsWhenHtpasswdFileMissing() throws Exception {
+    cluster = ClusterFixture.bareBuilder(dirTestWatcher)
+      .clusterSize(3)
+      .configProperty(ExecConstants.ALLOW_LOOPBACK_ADDRESS_BINDING, true)
+      .configProperty(ExecConstants.USER_AUTHENTICATION_ENABLED, true)
+      .configProperty(ExecConstants.USER_AUTHENTICATOR_IMPL, "htpasswd")
+      .configProperty(ExecConstants.HTPASSWD_AUTHENTICATOR_PATH, "/nonexistant-file")
+      .build();
+    tryCredentials("bob", "bob", cluster, false);
+  }
+
+  @Test
+  public void detectsChanges() throws Exception {
+    String passwdContent = "alice:pass1\nbob:buzzkill\n";
+    setupCluster(passwdContent);
+
+    tryCredentials("alice", "pass1", cluster, true);
+    tryCredentials("alice", "pass2", cluster, false);
+    tryCredentials("bob", "buzzkill", cluster, true);
+    tryCredentials("bob", "yolo", cluster, false);
+
+    String passwdContent2 = "alice:pass2\nbob:yolo\n";
+    Files.write(tempPasswdFile.toPath(), passwdContent2.getBytes());
+
+    tryCredentials("alice", "pass1", cluster, false);
+    tryCredentials("alice", "pass2", cluster, true);
+    tryCredentials("bob", "buzzkill", cluster, false);
+    tryCredentials("bob", "yolo", cluster, true);
+
+    // Invalid file is treated as empty
+    String passwdContent3 = "invalid file";
+    Files.write(tempPasswdFile.toPath(), passwdContent3.getBytes());
+
+    tryCredentials("alice", "pass1", cluster, false);
+    tryCredentials("alice", "pass2", cluster, false);
+
+    // Missing file is treated as empty
+    Files.delete(tempPasswdFile.toPath());
+
+    tryCredentials("alice", "pass1", cluster, false);
+    tryCredentials("alice", "pass2", cluster, false);
+
+  }
+
+  private static void tryCredentials(String user, String password, ClusterFixture cluster, boolean shouldSucceed) throws Exception {
+    try {
+      ClientFixture client = cluster.clientBuilder()
+        .property(DrillProperties.USER, user)
+        .property(DrillProperties.PASSWORD, password)
+        .build();
+
+      // Run few queries using the new client
+      List<String> queries = Arrays.asList(
+        "SHOW SCHEMAS",
+        "USE INFORMATION_SCHEMA",
+        "SHOW TABLES",
+        "SELECT * FROM INFORMATION_SCHEMA.`TABLES` WHERE TABLE_NAME LIKE 'COLUMNS'",
+        "SELECT * FROM cp.`region.json` LIMIT 5");
+
+      for (String query : queries) {
+        client.queryBuilder().sql(query).run();
+      }
+
+      if (!shouldSucceed) {
+        fail("Expected connect to fail because of incorrect username / password combination, but it succeeded");
+      }
+    } catch (IllegalStateException e) {
+      if (shouldSucceed) {
+        throw e;
+      }
+    }
+  }
+
+}