You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2020/11/20 14:20:23 UTC

[GitHub] [accumulo] BukrosSzabolcs opened a new pull request #1798: Hash upgrade

BukrosSzabolcs opened a new pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798


   Switch to crypt(3) hashing when storing user passwords in zk
   Extending authentication to recognize users with outdated hash and after a warning at startup update them on successful authentication.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] BukrosSzabolcs commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r530243983



##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
##########
@@ -103,7 +102,8 @@ private static SystemToken get(String instanceID, SiteConfiguration siteConfig)
       byte[] confChecksum;
       MessageDigest md;
       try {
-        md = MessageDigest.getInstance(Constants.PW_HASH_ALGORITHM);
+        String hashAlgorithm = siteConfig.get(Property.SYSTEM_TOKEN_HASH_TYPE);
+        md = MessageDigest.getInstance(hashAlgorithm);

Review comment:
       I would prefer to leave this as it is for now and create a separate issue for the follow up changes and related discussion.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] milleruntime commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
milleruntime commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r533742598



##########
File path: server/manager/src/main/java/org/apache/accumulo/master/Master.java
##########
@@ -1178,6 +1178,9 @@ boolean canSuspendTablets() {
       log.info("All metrics modules registered");
     }
 
+    // checking stored user hashes if any of them uses an outdated algorithm
+    security.validateStoredUserCreditentials();

Review comment:
       Good point.  And we can easily move the check somewhere else in later versions.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] BukrosSzabolcs commented on pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#issuecomment-740500983


   @ctubbsii The outstanding request was resolved and got the approval. Have you had time to do some manual testing?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] milleruntime commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
milleruntime commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r533738392



##########
File path: server/manager/src/main/java/org/apache/accumulo/master/Master.java
##########
@@ -1178,6 +1178,9 @@ boolean canSuspendTablets() {
       log.info("All metrics modules registered");
     }
 
+    // checking stored user hashes if any of them uses an outdated algorithm
+    security.validateStoredUserCreditentials();

Review comment:
       I am wondering if this is the best place to validate all user creds.  On one hand, it may be better to do this for some situations every time Master starts up but on the other, it may be sufficient and more efficient to only check once on upgrade.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] ctubbsii commented on pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#issuecomment-740740073


   > This looks great to me. I did some manual testing, and everything works as expected. I do think it would be useful to see an INFO (maybe DEBUG? any opinions here?) message when each user's credentials are upgraded, though.
   
   Don't worry about this one. I will add a commit to add a log message for the upgrade action before merging. Everything here looks great! I will merge it soon. Thanks, @BukrosSzabolcs !
   
   > Also, I noticed some strange "denied" messages for "performSystemAction" when a user successfully authenticates. I don't know if that's new with this PR or not. I'm going to see if I can track it down. If it's pre-existing, then I wouldn't worry about it.
   
   It looks like this is a pre-existing issue. Don't worry about it.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] ctubbsii commented on pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#issuecomment-741078149


   Thank you so much for your contribution! I'm really happy we were finally able to make this change!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] BukrosSzabolcs commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r530246165



##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -54,6 +56,33 @@ public void initialize(ServerContext context) {
     ZKUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
   }
 
+  /**
+   * Checks stored users and logs a warning containing the ones with outdated hashes.
+   */
+  public boolean hasOutdatedHashes() {

Review comment:
       I considered that too, but kept it like this because we are checking for the outdated format, and while not being outdated technically means it's the current one, the name still feels misleading.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] BukrosSzabolcs commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r530249701



##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java
##########
@@ -47,6 +51,7 @@
 class ZKSecurityTool {
   private static final Logger log = LoggerFactory.getLogger(ZKSecurityTool.class);
   private static final int SALT_LENGTH = 8;
+  private static final Charset CRYPT_CHARSET = UTF_8;

Review comment:
       Sure, thanks for pointing it out.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] ctubbsii commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r533741183



##########
File path: server/manager/src/main/java/org/apache/accumulo/master/Master.java
##########
@@ -1178,6 +1178,9 @@ boolean canSuspendTablets() {
       log.info("All metrics modules registered");
     }
 
+    // checking stored user hashes if any of them uses an outdated algorithm
+    security.validateStoredUserCreditentials();

Review comment:
       I was wondering that, too, but I kind of want users to be annoyed by it more than once, so that they actually log users in to upgrade the hashes. I was also thinking maybe the other route, and ~~nagging~~ *logging* once every few days or once a week on a running system. Once at master startup is probably a good compromise for now, though. We can get more aggressive in future releases, before eventually forcing it at upgrade.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] ctubbsii commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r529827988



##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -54,6 +56,33 @@ public void initialize(ServerContext context) {
     ZKUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
   }
 
+  /**
+   * Checks stored users and logs a warning containing the ones with outdated hashes.
+   */
+  public boolean hasOutdatedHashes() {

Review comment:
       If it makes more readable code in the callers (fewer negations), you could also flip the return values, and call this method `areHashesCurrent()` or similar.

##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java
##########
@@ -47,6 +51,7 @@
 class ZKSecurityTool {
   private static final Logger log = LoggerFactory.getLogger(ZKSecurityTool.class);
   private static final int SALT_LENGTH = 8;
+  private static final Charset CRYPT_CHARSET = UTF_8;

Review comment:
       Could inline this constant, to avoid the extra `CRYPT_CHARSET` variable and corresponding Charset import.

##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
##########
@@ -103,7 +102,8 @@ private static SystemToken get(String instanceID, SiteConfiguration siteConfig)
       byte[] confChecksum;
       MessageDigest md;
       try {
-        md = MessageDigest.getInstance(Constants.PW_HASH_ALGORITHM);
+        String hashAlgorithm = siteConfig.get(Property.SYSTEM_TOKEN_HASH_TYPE);
+        md = MessageDigest.getInstance(hashAlgorithm);

Review comment:
       Since system credentials are not serialized anywhere, but we do want them to be a strong hash, I think it might be best to make use of crypt(3) here also, so we're using the default best hash available in the commons-codec library.
   
   We can use a fixed salt for this. We could make the salt configurable or based on `instance.secret`, but there's no additional security added by doing so, since `instance.secret` is configurable and is already included in the message digest. So, a fixed salt would suffice here. If you're not interested in pursuing this, here, it can be done as a follow-on issue. Just let me know if that's the case, so I can create a new issue for it.
   
   As you've identified, the one benefit to making it configurable and preserving the current one is to support rolling restart. However, we don't support rolling restarts very well already, and certainly not across major or minor releases, and we've likely already broken compatibility of that sort during 2.1's development (or will, if we upgrade Thrift). Also, I think having a strong has for the system user is important enough to break that, even if it did otherwise work. So, I'm inclined to favor not making it configurable.

##########
File path: core/src/main/java/org/apache/accumulo/core/conf/Property.java
##########
@@ -83,6 +83,14 @@
   RPC_SASL_QOP("rpc.sasl.qop", "auth", PropertyType.STRING,
       "The quality of protection to be used with SASL. Valid values are 'auth', 'auth-int',"
           + " and 'auth-conf'"),
+  /**
+   * @since 2.1.0
+   */
+  SYSTEM_TOKEN_HASH_TYPE("system.token.hash.type", Constants.PW_HASH_ALGORITHM_OUTDATED,

Review comment:
       See my other comment about not making this configurable. However, since it is configurable, it must be the same across all servers, or they won't be able to talk to each other. In general, that would mean making this an `instance.*` property. However, since the way we enforce these is through the same-ness of the system credentials, if this is different across the cluster, then they won't be able to talk to each other anyway. The other property prefix we use for system-wide properties that can be different are `general.*`.

##########
File path: core/src/main/java/org/apache/accumulo/core/file/rfile/VisMetricsGatherer.java
##########
@@ -136,12 +136,12 @@ public void printMetrics(boolean hash, String metricWord, PrintStream out) {
         if (hash) {
           String encodedKey = "";
           try {
-            byte[] encodedBytes = MessageDigest.getInstance(Constants.PW_HASH_ALGORITHM)
+            byte[] encodedBytes = MessageDigest.getInstance(Constants.NON_CRYPTO_USE_HASH_ALGORITHM)

Review comment:
       Since this use of the digest is specific to this specific metric utility's serialization, we can probably just use a local constant, rather than one in `Constants.java` (which I'd personally like to phase out over time).

##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java
##########
@@ -99,6 +102,24 @@ public static boolean checkPass(byte[] password, byte[] zkData) {
     return saltedHash; // contains salt+hash(password+salt)
   }
 
+  public static byte[] createPass(byte[] password) throws AccumuloException {
+    // we rely on default algorithm and hash length (SHA-512 and 8 byte)
+    String cryptHash = Crypt.crypt(password);
+    return cryptHash.getBytes(CRYPT_CHARSET);
+  }
+
+  public static boolean checkCryptPass(byte[] password, byte[] zkData) {
+    String zkDataString = new String(zkData, CRYPT_CHARSET);
+    String cryptHash;
+    try {
+      cryptHash = Crypt.crypt(password, zkDataString);
+    } catch (IllegalArgumentException e) {
+      log.error("Unrecognized hash format", e);
+      return false;
+    }
+    return MessageDigest.isEqual(zkData, cryptHash.getBytes(CRYPT_CHARSET));

Review comment:
       Okay. Fair enough.

##########
File path: shell/src/main/java/org/apache/accumulo/shell/commands/GetSplitsCommand.java
##########
@@ -113,7 +113,7 @@ private static String encode(final boolean encode, final Text text) {
   private static String obscuredTabletName(final KeyExtent extent) {
     MessageDigest digester;
     try {
-      digester = MessageDigest.getInstance(Constants.PW_HASH_ALGORITHM);
+      digester = MessageDigest.getInstance(Constants.NON_CRYPTO_USE_HASH_ALGORITHM);

Review comment:
       It looks like we have this exact same algorithm for creating obscured tablet identifiers implemented in at least two places. Creating a new issue (#1805) to address that separately.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] BukrosSzabolcs commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r528868213



##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -52,6 +52,23 @@ public void initialize(ServerContext context) {
     this.context = context;
     zooCache = new ZooCache(context.getZooReaderWriter(), null);
     ZKUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
+    checkOutdatedHashes();
+  }
+
+  private void checkOutdatedHashes() {
+    try {
+      listUsers().forEach(user -> {
+        String zpath = ZKUserPath + "/" + user;
+        byte[] zkData = zooCache.get(zpath);
+        if (ZKSecurityTool.isOutdatedPass(zkData)) {
+          log.warn("Found user(s) with outdated password hash. These will be re-hashed"
+              + " on successful authentication.");

Review comment:
       I'm never sure exactly how much detail is too much when it comes to logging security features. I can add an affected user count.
   I'm not sure the operator has to or should do anything, the update is automatic. But with their current tools they could replicate what the code does and could go through the users zNode checking hash lengths. Alternately I could list the affected user principals in the warning but that feels unsafe. Do you have something in mind?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] ctubbsii commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r528921037



##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -52,6 +52,23 @@ public void initialize(ServerContext context) {
     this.context = context;
     zooCache = new ZooCache(context.getZooReaderWriter(), null);
     ZKUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
+    checkOutdatedHashes();
+  }
+
+  private void checkOutdatedHashes() {
+    try {
+      listUsers().forEach(user -> {
+        String zpath = ZKUserPath + "/" + user;
+        byte[] zkData = zooCache.get(zpath);
+        if (ZKSecurityTool.isOutdatedPass(zkData)) {
+          log.warn("Found user(s) with outdated password hash. These will be re-hashed"
+              + " on successful authentication.");
+          return;
+        }
+      });
+    } catch (NullPointerException e) {
+      // initializeSecurity was not called yet, there could be no outdated passwords stored

Review comment:
       It looks like initialize itself is already an edge case... see my comment at https://github.com/apache/accumulo/pull/1798#discussion_r528903518




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] milleruntime commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
milleruntime commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r533653675



##########
File path: core/src/main/java/org/apache/accumulo/core/file/rfile/VisMetricsGatherer.java
##########
@@ -47,6 +46,7 @@
  */
 public class VisMetricsGatherer
     implements MetricsGatherer<Map<String,ArrayList<VisibilityMetric>>> {
+  static final String KEY_HASH_ALGORITHM = "SHA-256";

Review comment:
       This is an odd one.  I am not sure what we are using the HASH to encrypt here.  If it is just to prevent leaking data then I _think_ it would be OK to use the NON_CRYPTO_USE_HASH_ALGORITHM




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] ctubbsii commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r528858607



##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java
##########
@@ -99,6 +102,24 @@ public static boolean checkPass(byte[] password, byte[] zkData) {
     return saltedHash; // contains salt+hash(password+salt)
   }
 
+  public static byte[] createPass(byte[] password) throws AccumuloException {
+    // we rely on default algorithm and hash length (SHA-512 and 8 byte)
+    String cryptHash = Crypt.crypt(password);
+    return cryptHash.getBytes(CRYPT_CHARSET);
+  }
+
+  public static boolean checkCryptPass(byte[] password, byte[] zkData) {
+    String zkDataString = new String(zkData, CRYPT_CHARSET);
+    String cryptHash;
+    try {
+      cryptHash = Crypt.crypt(password, zkDataString);
+    } catch (IllegalArgumentException e) {
+      log.error("Unrecognized hash format", e);
+      return false;
+    }
+    return MessageDigest.isEqual(zkData, cryptHash.getBytes(CRYPT_CHARSET));

Review comment:
       What's the advantage of using `MessageDigest.isEqual`, vs. comparing as Strings or comparing as the UTF-8 byte-array?

##########
File path: server/base/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java
##########
@@ -88,14 +90,40 @@ public void testTableConversion() {
 
   @Test
   public void testEncryption() {
+    byte[] rawPass = "myPassword".getBytes(Charset.forName("UTF-8"));
+    byte[] storedBytes;
+    try {
+      storedBytes = ZKSecurityTool.createPass(rawPass.clone());
+      assertTrue(ZKSecurityTool.checkCryptPass(rawPass.clone(), storedBytes));
+    } catch (AccumuloException e) {
+      log.error("{}", e.getMessage(), e);
+      fail();
+    }

Review comment:
       You can simplify this JUnit test by just throwing this out of the method. JUnit will show the details of the exception in its logs.

##########
File path: test/src/main/java/org/apache/accumulo/test/ZKAuthenticatorIT.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.accumulo.test;
+
+import static org.junit.Assert.assertTrue;
+
+import java.nio.charset.Charset;
+
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.accumulo.server.security.handler.ZKAuthenticator;
+import org.junit.Test;
+
+public class ZKAuthenticatorIT extends AccumuloClusterHarness {

Review comment:
       If we can avoid running a full ZK instance using Mini, I think we should. I think ZKAuthenticator can be tested fully using unit tests with mocking, without a need to start up an instance of a cluster.

##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -52,6 +52,23 @@ public void initialize(ServerContext context) {
     this.context = context;
     zooCache = new ZooCache(context.getZooReaderWriter(), null);
     ZKUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
+    checkOutdatedHashes();
+  }
+
+  private void checkOutdatedHashes() {
+    try {
+      listUsers().forEach(user -> {
+        String zpath = ZKUserPath + "/" + user;
+        byte[] zkData = zooCache.get(zpath);
+        if (ZKSecurityTool.isOutdatedPass(zkData)) {
+          log.warn("Found user(s) with outdated password hash. These will be re-hashed"
+              + " on successful authentication.");
+          return;
+        }
+      });
+    } catch (NullPointerException e) {
+      // initializeSecurity was not called yet, there could be no outdated passwords stored

Review comment:
       Could also avoid this by checking if the zk node exists first.

##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -52,6 +52,23 @@ public void initialize(ServerContext context) {
     this.context = context;
     zooCache = new ZooCache(context.getZooReaderWriter(), null);
     ZKUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
+    checkOutdatedHashes();
+  }
+
+  private void checkOutdatedHashes() {
+    try {
+      listUsers().forEach(user -> {
+        String zpath = ZKUserPath + "/" + user;
+        byte[] zkData = zooCache.get(zpath);
+        if (ZKSecurityTool.isOutdatedPass(zkData)) {
+          log.warn("Found user(s) with outdated password hash. These will be re-hashed"
+              + " on successful authentication.");

Review comment:
       One option is to provide the user name during these warnings. Another is to provide single warning, rather than one per user, and then provide some other mechanism to list outdated entries.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] ctubbsii commented on pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#issuecomment-740604215


   > @ctubbsii The outstanding request was resolved and got the approval. Have you had time to do some manual testing?
   
   Not yet. This is my priority today.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] ctubbsii commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r533686689



##########
File path: core/src/main/java/org/apache/accumulo/core/file/rfile/VisMetricsGatherer.java
##########
@@ -47,6 +46,7 @@
  */
 public class VisMetricsGatherer
     implements MetricsGatherer<Map<String,ArrayList<VisibilityMetric>>> {
+  static final String KEY_HASH_ALGORITHM = "SHA-256";

Review comment:
       > This is an odd one. I am not sure what we are using the HASH to encrypt here. If it is just to prevent leaking data then I _think_ it would be OK to use the NON_CRYPTO_USE_HASH_ALGORITHM
   
   @milleruntime I think this was moved here at my request to keep serialization-related constants local to the serialization code, rather than to rely on some global constant, which could have a multitude of uses. I think it's fine here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] Manno15 commented on pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
Manno15 commented on pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#issuecomment-733812669


   They all seemed to pass this time. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] BukrosSzabolcs commented on pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#issuecomment-733791276


   The test failure looks unrelated. How can I re-trigger the tests?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] BukrosSzabolcs commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r530396137



##########
File path: core/src/main/java/org/apache/accumulo/core/conf/Property.java
##########
@@ -83,6 +83,14 @@
   RPC_SASL_QOP("rpc.sasl.qop", "auth", PropertyType.STRING,
       "The quality of protection to be used with SASL. Valid values are 'auth', 'auth-int',"
           + " and 'auth-conf'"),
+  /**
+   * @since 2.1.0
+   */
+  SYSTEM_TOKEN_HASH_TYPE("system.token.hash.type", Constants.PW_HASH_ALGORITHM_OUTDATED,

Review comment:
       Thank you for the suggestions. Adding the `instance.*` prefix.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] BukrosSzabolcs commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r530373346



##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -116,6 +133,31 @@ public void createUser(String principal, AuthenticationToken token)
     }
   }
 
+  /**
+   * Creates user with outdated password hash for testing
+   *
+   * @deprecated since 2.1.0, only present for testing DO NOT USE!
+   */

Review comment:
       Just realized the only reason for this method was that the IT tests required it, but with reworking those to unit tests this is no longer used. So I'll just remove this.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] ctubbsii commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r530539938



##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
##########
@@ -103,7 +102,8 @@ private static SystemToken get(String instanceID, SiteConfiguration siteConfig)
       byte[] confChecksum;
       MessageDigest md;
       try {
-        md = MessageDigest.getInstance(Constants.PW_HASH_ALGORITHM);
+        String hashAlgorithm = siteConfig.get(Property.SYSTEM_TOKEN_HASH_TYPE);
+        md = MessageDigest.getInstance(hashAlgorithm);

Review comment:
       No problem. I created #1810 to deal with it as a follow-on issue.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] BukrosSzabolcs commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r529726393



##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -52,6 +52,23 @@ public void initialize(ServerContext context) {
     this.context = context;
     zooCache = new ZooCache(context.getZooReaderWriter(), null);
     ZKUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
+    checkOutdatedHashes();
+  }
+
+  private void checkOutdatedHashes() {
+    try {
+      listUsers().forEach(user -> {
+        String zpath = ZKUserPath + "/" + user;
+        byte[] zkData = zooCache.get(zpath);
+        if (ZKSecurityTool.isOutdatedPass(zkData)) {
+          log.warn("Found user(s) with outdated password hash. These will be re-hashed"
+              + " on successful authentication.");

Review comment:
       Added the check to master startup through SecurityOperation.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] BukrosSzabolcs commented on pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#issuecomment-740749767


   Thanks for the feedback and the help @ctubbsii, @busbey !


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] BukrosSzabolcs commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r529742636



##########
File path: test/src/main/java/org/apache/accumulo/test/ZKAuthenticatorIT.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.accumulo.test;
+
+import static org.junit.Assert.assertTrue;
+
+import java.nio.charset.Charset;
+
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.accumulo.server.security.handler.ZKAuthenticator;
+import org.junit.Test;
+
+public class ZKAuthenticatorIT extends AccumuloClusterHarness {

Review comment:
       It can. I'm rewriting it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] BukrosSzabolcs commented on pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#issuecomment-733835680


   @Manno15 Thanks for your help!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] busbey commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
busbey commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r530376166



##########
File path: core/src/main/java/org/apache/accumulo/core/conf/Property.java
##########
@@ -83,6 +83,14 @@
   RPC_SASL_QOP("rpc.sasl.qop", "auth", PropertyType.STRING,
       "The quality of protection to be used with SASL. Valid values are 'auth', 'auth-int',"
           + " and 'auth-conf'"),
+  /**
+   * @since 2.1.0
+   */
+  SYSTEM_TOKEN_HASH_TYPE("system.token.hash.type", Constants.PW_HASH_ALGORITHM_OUTDATED,

Review comment:
       I personally think `instance` seems an appropriate prefix, so it would be `instance.system.token.hash.type`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] busbey commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
busbey commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r528697020



##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -52,6 +52,23 @@ public void initialize(ServerContext context) {
     this.context = context;
     zooCache = new ZooCache(context.getZooReaderWriter(), null);
     ZKUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
+    checkOutdatedHashes();
+  }
+
+  private void checkOutdatedHashes() {
+    try {
+      listUsers().forEach(user -> {
+        String zpath = ZKUserPath + "/" + user;
+        byte[] zkData = zooCache.get(zpath);
+        if (ZKSecurityTool.isOutdatedPass(zkData)) {
+          log.warn("Found user(s) with outdated password hash. These will be re-hashed"
+              + " on successful authentication.");
+          return;
+        }
+      });
+    } catch (NullPointerException e) {
+      // initializeSecurity was not called yet, there could be no outdated passwords stored

Review comment:
       log a DEBUG message with these details.

##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -180,18 +222,43 @@ public boolean authenticateUser(String principal, AuthenticationToken token)
     if (!(token instanceof PasswordToken))
       throw new AccumuloSecurityException(principal, SecurityErrorCode.INVALID_TOKEN);
     PasswordToken pt = (PasswordToken) token;
-    byte[] pass;
+    byte[] zkData;
     String zpath = ZKUserPath + "/" + principal;
-    pass = zooCache.get(zpath);
-    boolean result = ZKSecurityTool.checkPass(pt.getPassword(), pass);
+    zkData = zooCache.get(zpath);
+    boolean result = authenticateUser(principal, pt, zkData);
     if (!result) {
       zooCache.clear(zpath);
-      pass = zooCache.get(zpath);
-      result = ZKSecurityTool.checkPass(pt.getPassword(), pass);
+      zkData = zooCache.get(zpath);
+      result = authenticateUser(principal, pt, zkData);
     }
     return result;
   }
 
+  private boolean authenticateUser(String principal, PasswordToken pt, byte[] zkData) {
+    if (zkData == null) {
+      return false;
+    }
+
+    // if the hash does not match the outdated format use Crypt to verify it
+    if (!ZKSecurityTool.isOutdatedPass(zkData)) {
+      return ZKSecurityTool.checkCryptPass(pt.getPassword(), zkData);
+    }
+
+    if (!ZKSecurityTool.checkPass(pt.getPassword(), zkData)) {
+      // if password does not match we are done
+      return false;
+    }
+
+    // if the password is correct we have to update the stored hash with new algorithm
+    try {
+      changePassword(principal, pt);
+      return true;
+    } catch (AccumuloSecurityException e) {
+      log.error("Failed to update hashed user password for user: {}", principal, e);
+    }
+    return false;

Review comment:
       if we fail to update the password for some reason (like a transient zk write failure), at this point shouldn't we still return that they correctly authenticated?

##########
File path: core/src/main/java/org/apache/accumulo/core/Constants.java
##########
@@ -99,7 +99,8 @@
   public static final long SCANNER_DEFAULT_READAHEAD_THRESHOLD = 3L;
 
   // Security configuration
-  public static final String PW_HASH_ALGORITHM = "SHA-256";
+  public static final String PW_HASH_ALGORITHM = "SHA-512";

Review comment:
       We still need this constant updated because we rely on it for hashing the system credentials? won't that prevent a rolling upgrade?
   
   Could we have system credentials fall back to SHA-256 with a warning? or require a configurable flag to switch it?

##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java
##########
@@ -47,6 +49,7 @@
 class ZKSecurityTool {
   private static final Logger log = LoggerFactory.getLogger(ZKSecurityTool.class);
   private static final int SALT_LENGTH = 8;
+  private static final Charset CRYPT_CHARSET = Charset.forName("UTF-8");

Review comment:
       add a comment about why a new Charset instead of relying on `StandardCharsets.UTF_8`

##########
File path: core/src/main/java/org/apache/accumulo/core/Constants.java
##########
@@ -99,7 +99,8 @@
   public static final long SCANNER_DEFAULT_READAHEAD_THRESHOLD = 3L;
 
   // Security configuration
-  public static final String PW_HASH_ALGORITHM = "SHA-256";
+  public static final String PW_HASH_ALGORITHM = "SHA-512";

Review comment:
       changing this is also going to change some non-security uses, e.g. we optionally use it to obscure values printed from rfile metrics gathering. We'll need to enumerate these and release note the change in behavior. (or we could make something like a `NON_CRYPTO_USE_HASH_ALGORITHM` that we keep as SHA-256)

##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -116,6 +133,31 @@ public void createUser(String principal, AuthenticationToken token)
     }
   }
 
+  /**
+   * Creates user with outdated password hash for testing
+   *
+   * @deprecated since 2.1.0, only present for testing DO NOT USE!
+   */

Review comment:
       could we log a WARN message that this method has been used? that way it would show up in operator logs should we mistakenly use it in a non-test context.

##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -52,6 +52,23 @@ public void initialize(ServerContext context) {
     this.context = context;
     zooCache = new ZooCache(context.getZooReaderWriter(), null);
     ZKUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
+    checkOutdatedHashes();
+  }
+
+  private void checkOutdatedHashes() {
+    try {
+      listUsers().forEach(user -> {
+        String zpath = ZKUserPath + "/" + user;
+        byte[] zkData = zooCache.get(zpath);
+        if (ZKSecurityTool.isOutdatedPass(zkData)) {
+          log.warn("Found user(s) with outdated password hash. These will be re-hashed"
+              + " on successful authentication.");

Review comment:
       since we aren't including any details about which user(s) are impacted, I'd rather we not get a WARN for each user. could we move this to after we finish iterating with a summary of how many users?
   
   If an operator needed to move towards eliminating these warn messages, how would they get the list of users that need to authenticate to the system?

##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java
##########
@@ -56,16 +59,26 @@
     return salt;
   }
 
+  // only present for testing DO NOT USE!
+  @Deprecated(since = "2.1.0")
+  static byte[] createOutdatedPass(byte[] password) throws AccumuloException {
+    byte[] salt = generateSalt();
+    try {
+      return convertPass(password, salt);
+    } catch (NoSuchAlgorithmException e) {
+      log.error("Count not create hashed password", e);
+      throw new AccumuloException("Count not create hashed password", e);
+    }
+  }
+
   private static byte[] hash(byte[] raw) throws NoSuchAlgorithmException {
-    MessageDigest md = MessageDigest.getInstance(Constants.PW_HASH_ALGORITHM);
+    MessageDigest md = MessageDigest.getInstance(Constants.PW_HASH_ALGORITHM_OUTDATED);
     md.update(raw);
     return md.digest();
   }
 
+  @Deprecated(since = "2.1.0")
   public static boolean checkPass(byte[] password, byte[] zkData) {
-    if (zkData == null)
-      return false;
-

Review comment:
       just to make sure I understand the reasoning here, the removal of this check is because ZKSecurityTool is package private and all current calls ensure zkData isn't null?
   
   if that's correct please add javadocs that say zkData can't be null. an alternative is to leave the check in place and rely on the jit to optimize it away.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] milleruntime commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
milleruntime commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r533733535



##########
File path: core/src/main/java/org/apache/accumulo/core/file/rfile/VisMetricsGatherer.java
##########
@@ -47,6 +46,7 @@
  */
 public class VisMetricsGatherer
     implements MetricsGatherer<Map<String,ArrayList<VisibilityMetric>>> {
+  static final String KEY_HASH_ALGORITHM = "SHA-256";

Review comment:
       Ah I see.  It makes me wonder why are we even bother hashing it at all for display purposes.  Why not just print gobbledygook or just don't print it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] ctubbsii edited a comment on pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
ctubbsii edited a comment on pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#issuecomment-740740073


   > This looks great to me. I did some manual testing, and everything works as expected. I do think it would be useful to see an INFO (maybe DEBUG? any opinions here?) message when each user's credentials are upgraded, though.
   
   Don't worry about this one. I will add a commit to add a log message for the upgrade action before merging. Everything here looks great! I will merge it soon. Thanks, @BukrosSzabolcs !
   
   > Also, I noticed some strange "denied" messages for "performSystemAction" when a user successfully authenticates. I don't know if that's new with this PR or not. I'm going to see if I can track it down. If it's pre-existing, then I wouldn't worry about it.
   
   It looks like this is a pre-existing issue. Don't worry about it. (EDIT: fixed in #1827 )
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] ctubbsii commented on pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#issuecomment-741077616


   @BukrosSzabolcs I ran into a problem while testing, because I made a mistake on #1827 , but I reverted that, and was able to finally merge this change in (with my added debug log statement, as discussed).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] ctubbsii commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r528921037



##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -52,6 +52,23 @@ public void initialize(ServerContext context) {
     this.context = context;
     zooCache = new ZooCache(context.getZooReaderWriter(), null);
     ZKUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
+    checkOutdatedHashes();
+  }
+
+  private void checkOutdatedHashes() {
+    try {
+      listUsers().forEach(user -> {
+        String zpath = ZKUserPath + "/" + user;
+        byte[] zkData = zooCache.get(zpath);
+        if (ZKSecurityTool.isOutdatedPass(zkData)) {
+          log.warn("Found user(s) with outdated password hash. These will be re-hashed"
+              + " on successful authentication.");
+          return;
+        }
+      });
+    } catch (NullPointerException e) {
+      // initializeSecurity was not called yet, there could be no outdated passwords stored

Review comment:
       It looks like initialize itself is already an edge case... see my comment at https://github.com/apache/accumulo/pull/1798/files/4c8fedce90487c40ec4b740bc98dc181177cc71f#r528903518




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] busbey commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
busbey commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r530367801



##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -116,6 +133,31 @@ public void createUser(String principal, AuthenticationToken token)
     }
   }
 
+  /**
+   * Creates user with outdated password hash for testing
+   *
+   * @deprecated since 2.1.0, only present for testing DO NOT USE!
+   */

Review comment:
       yeah that sounds great.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] ctubbsii merged pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
ctubbsii merged pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] ctubbsii commented on pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#issuecomment-740647642


   @BukrosSzabolcs I added a commit to merge the main branch onto this, to get it up-to-date with current main, and to resolve compiler warnings, but I'm still testing. I should be done today.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] ctubbsii commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r529815899



##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -52,6 +52,23 @@ public void initialize(ServerContext context) {
     this.context = context;
     zooCache = new ZooCache(context.getZooReaderWriter(), null);
     ZKUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
+    checkOutdatedHashes();
+  }
+
+  private void checkOutdatedHashes() {
+    try {
+      listUsers().forEach(user -> {
+        String zpath = ZKUserPath + "/" + user;
+        byte[] zkData = zooCache.get(zpath);
+        if (ZKSecurityTool.isOutdatedPass(zkData)) {
+          log.warn("Found user(s) with outdated password hash. These will be re-hashed"
+              + " on successful authentication.");

Review comment:
       @BukrosSzabolcs Feel free to mark the conversation as "Resolved" if you've taken steps to address the comments. It cleans up the interface and makes it easier for subsequent reviews. :smiley_cat: 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] milleruntime commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
milleruntime commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r533700645



##########
File path: core/src/main/java/org/apache/accumulo/core/file/rfile/VisMetricsGatherer.java
##########
@@ -47,6 +46,7 @@
  */
 public class VisMetricsGatherer
     implements MetricsGatherer<Map<String,ArrayList<VisibilityMetric>>> {
+  static final String KEY_HASH_ALGORITHM = "SHA-256";

Review comment:
       I don't think this is serializing anything.  I thought this method was only used by the `rfile-info` command. The only use i can find of `printMetircs()` is here: https://github.com/apache/accumulo/blob/6a74b4667e3bd33e34b5262c5dd8ea64167fb657/core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java#L291




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] busbey commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
busbey commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r532874751



##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -180,18 +222,43 @@ public boolean authenticateUser(String principal, AuthenticationToken token)
     if (!(token instanceof PasswordToken))
       throw new AccumuloSecurityException(principal, SecurityErrorCode.INVALID_TOKEN);
     PasswordToken pt = (PasswordToken) token;
-    byte[] pass;
+    byte[] zkData;
     String zpath = ZKUserPath + "/" + principal;
-    pass = zooCache.get(zpath);
-    boolean result = ZKSecurityTool.checkPass(pt.getPassword(), pass);
+    zkData = zooCache.get(zpath);
+    boolean result = authenticateUser(principal, pt, zkData);
     if (!result) {
       zooCache.clear(zpath);
-      pass = zooCache.get(zpath);
-      result = ZKSecurityTool.checkPass(pt.getPassword(), pass);
+      zkData = zooCache.get(zpath);
+      result = authenticateUser(principal, pt, zkData);
     }
     return result;
   }
 
+  private boolean authenticateUser(String principal, PasswordToken pt, byte[] zkData) {
+    if (zkData == null) {
+      return false;
+    }
+
+    // if the hash does not match the outdated format use Crypt to verify it
+    if (!ZKSecurityTool.isOutdatedPass(zkData)) {
+      return ZKSecurityTool.checkCryptPass(pt.getPassword(), zkData);
+    }
+
+    if (!ZKSecurityTool.checkPass(pt.getPassword(), zkData)) {
+      // if password does not match we are done
+      return false;
+    }
+
+    // if the password is correct we have to update the stored hash with new algorithm
+    try {
+      changePassword(principal, pt);
+      return true;
+    } catch (AccumuloSecurityException e) {
+      log.error("Failed to update hashed user password for user: {}", principal, e);
+    }
+    return false;

Review comment:
       okay, sounds good. @BukrosSzabolcs I think you're fine to leave this as-is.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] ctubbsii commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r533711549



##########
File path: core/src/main/java/org/apache/accumulo/core/file/rfile/VisMetricsGatherer.java
##########
@@ -47,6 +46,7 @@
  */
 public class VisMetricsGatherer
     implements MetricsGatherer<Map<String,ArrayList<VisibilityMetric>>> {
+  static final String KEY_HASH_ALGORITHM = "SHA-256";

Review comment:
       That constant can be left inside this class, because it's not being used as some sort of overall crypto strategy that necessitates it be aligned with how the other constant is being used... it's just an arbitrary choice for this particular visibility metric output format (or "serialization algorithm") and relates to nothing else. So, it can/should be a local constant held closer to the rest of that metric serialization algorithm. That's all I mean.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] ctubbsii commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r530537337



##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -180,18 +222,43 @@ public boolean authenticateUser(String principal, AuthenticationToken token)
     if (!(token instanceof PasswordToken))
       throw new AccumuloSecurityException(principal, SecurityErrorCode.INVALID_TOKEN);
     PasswordToken pt = (PasswordToken) token;
-    byte[] pass;
+    byte[] zkData;
     String zpath = ZKUserPath + "/" + principal;
-    pass = zooCache.get(zpath);
-    boolean result = ZKSecurityTool.checkPass(pt.getPassword(), pass);
+    zkData = zooCache.get(zpath);
+    boolean result = authenticateUser(principal, pt, zkData);
     if (!result) {
       zooCache.clear(zpath);
-      pass = zooCache.get(zpath);
-      result = ZKSecurityTool.checkPass(pt.getPassword(), pass);
+      zkData = zooCache.get(zpath);
+      result = authenticateUser(principal, pt, zkData);
     }
     return result;
   }
 
+  private boolean authenticateUser(String principal, PasswordToken pt, byte[] zkData) {
+    if (zkData == null) {
+      return false;
+    }
+
+    // if the hash does not match the outdated format use Crypt to verify it
+    if (!ZKSecurityTool.isOutdatedPass(zkData)) {
+      return ZKSecurityTool.checkCryptPass(pt.getPassword(), zkData);
+    }
+
+    if (!ZKSecurityTool.checkPass(pt.getPassword(), zkData)) {
+      // if password does not match we are done
+      return false;
+    }
+
+    // if the password is correct we have to update the stored hash with new algorithm
+    try {
+      changePassword(principal, pt);
+      return true;
+    } catch (AccumuloSecurityException e) {
+      log.error("Failed to update hashed user password for user: {}", principal, e);
+    }
+    return false;

Review comment:
       I don't feel very strongly about it. However, I do lean towards forcing the upgrade and rejecting if it can't be completed (for the reasons @busbey mentioned, as well as because I want to eventually ensure we can drop the code that supports reading the old hashes on some future upgrade).
   I think it's very unlikely we're going to fail here if we've already authenticated, unless ZK is having issues (in which case, we'll probably see other errors in the system). So, it may be moot. Either way would probably be fine.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] ctubbsii commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r533709719



##########
File path: core/src/main/java/org/apache/accumulo/core/file/rfile/VisMetricsGatherer.java
##########
@@ -47,6 +46,7 @@
  */
 public class VisMetricsGatherer
     implements MetricsGatherer<Map<String,ArrayList<VisibilityMetric>>> {
+  static final String KEY_HASH_ALGORITHM = "SHA-256";

Review comment:
       By serialization, I simply mean the serial form that we use for this particular display format.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] ctubbsii commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
ctubbsii commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r528903518



##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -52,6 +52,23 @@ public void initialize(ServerContext context) {
     this.context = context;
     zooCache = new ZooCache(context.getZooReaderWriter(), null);
     ZKUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
+    checkOutdatedHashes();
+  }
+
+  private void checkOutdatedHashes() {
+    try {
+      listUsers().forEach(user -> {
+        String zpath = ZKUserPath + "/" + user;
+        byte[] zkData = zooCache.get(zpath);
+        if (ZKSecurityTool.isOutdatedPass(zkData)) {
+          log.warn("Found user(s) with outdated password hash. These will be re-hashed"
+              + " on successful authentication.");

Review comment:
       The list of users in the server logs is okay. However, I just realized... this method is only ever called when we're starting from an empty user database (on initialize... or re-initialize), so there shouldn't be any matching the old hashes anyway at the point this code is run. Or did I miss something?
   
   I do think maybe it would be worth the master server checking to see if any users are using the old hashing, and listing them in the server logs on startup, as an upgrade warning. In future, we can turn that into an upgrade blocker (so we can eventually drop the code that supports reading the old hashes).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] busbey commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
busbey commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r530373941



##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -180,18 +222,43 @@ public boolean authenticateUser(String principal, AuthenticationToken token)
     if (!(token instanceof PasswordToken))
       throw new AccumuloSecurityException(principal, SecurityErrorCode.INVALID_TOKEN);
     PasswordToken pt = (PasswordToken) token;
-    byte[] pass;
+    byte[] zkData;
     String zpath = ZKUserPath + "/" + principal;
-    pass = zooCache.get(zpath);
-    boolean result = ZKSecurityTool.checkPass(pt.getPassword(), pass);
+    zkData = zooCache.get(zpath);
+    boolean result = authenticateUser(principal, pt, zkData);
     if (!result) {
       zooCache.clear(zpath);
-      pass = zooCache.get(zpath);
-      result = ZKSecurityTool.checkPass(pt.getPassword(), pass);
+      zkData = zooCache.get(zpath);
+      result = authenticateUser(principal, pt, zkData);
     }
     return result;
   }
 
+  private boolean authenticateUser(String principal, PasswordToken pt, byte[] zkData) {
+    if (zkData == null) {
+      return false;
+    }
+
+    // if the hash does not match the outdated format use Crypt to verify it
+    if (!ZKSecurityTool.isOutdatedPass(zkData)) {
+      return ZKSecurityTool.checkCryptPass(pt.getPassword(), zkData);
+    }
+
+    if (!ZKSecurityTool.checkPass(pt.getPassword(), zkData)) {
+      // if password does not match we are done
+      return false;
+    }
+
+    // if the password is correct we have to update the stored hash with new algorithm
+    try {
+      changePassword(principal, pt);
+      return true;
+    } catch (AccumuloSecurityException e) {
+      log.error("Failed to update hashed user password for user: {}", principal, e);
+    }
+    return false;

Review comment:
       I think this comes down to wether we want to fail open or closed for updating to a more secure hash. I personally prefer failing open in this case -- essentially if there's a problem writing to ZK we did not consider here then the system will keep working as before the upgrade but with a persistent log message an operator could use to go fix things.
   
   I could see at least one argument for failing closed: a person or system not being able to talk to Accumulo will be a very loud noise that will ensure whatever is preventing us from storing things more securely gets handled. Presuming secure hashing of password data is an important system requirement in a particular Accumulo deployment that is a good thing.
   
   Does anyone else feel strongly?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [accumulo] BukrosSzabolcs commented on a change in pull request #1798: Hash upgrade

Posted by GitBox <gi...@apache.org>.
BukrosSzabolcs commented on a change in pull request #1798:
URL: https://github.com/apache/accumulo/pull/1798#discussion_r528898191



##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java
##########
@@ -47,6 +49,7 @@
 class ZKSecurityTool {
   private static final Logger log = LoggerFactory.getLogger(ZKSecurityTool.class);
   private static final int SALT_LENGTH = 8;
+  private static final Charset CRYPT_CHARSET = Charset.forName("UTF-8");

Review comment:
       Switching to `StandardCharsets.UTF_8`

##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -180,18 +222,43 @@ public boolean authenticateUser(String principal, AuthenticationToken token)
     if (!(token instanceof PasswordToken))
       throw new AccumuloSecurityException(principal, SecurityErrorCode.INVALID_TOKEN);
     PasswordToken pt = (PasswordToken) token;
-    byte[] pass;
+    byte[] zkData;
     String zpath = ZKUserPath + "/" + principal;
-    pass = zooCache.get(zpath);
-    boolean result = ZKSecurityTool.checkPass(pt.getPassword(), pass);
+    zkData = zooCache.get(zpath);
+    boolean result = authenticateUser(principal, pt, zkData);
     if (!result) {
       zooCache.clear(zpath);
-      pass = zooCache.get(zpath);
-      result = ZKSecurityTool.checkPass(pt.getPassword(), pass);
+      zkData = zooCache.get(zpath);
+      result = authenticateUser(principal, pt, zkData);
     }
     return result;
   }
 
+  private boolean authenticateUser(String principal, PasswordToken pt, byte[] zkData) {
+    if (zkData == null) {
+      return false;
+    }
+
+    // if the hash does not match the outdated format use Crypt to verify it
+    if (!ZKSecurityTool.isOutdatedPass(zkData)) {
+      return ZKSecurityTool.checkCryptPass(pt.getPassword(), zkData);
+    }
+
+    if (!ZKSecurityTool.checkPass(pt.getPassword(), zkData)) {
+      // if password does not match we are done
+      return false;
+    }
+
+    // if the password is correct we have to update the stored hash with new algorithm
+    try {
+      changePassword(principal, pt);
+      return true;
+    } catch (AccumuloSecurityException e) {
+      log.error("Failed to update hashed user password for user: {}", principal, e);
+    }
+    return false;

Review comment:
       My reasoning was that we are trying to re-hash the password if possible and on failure we are triggering retry and it should not consistently fail. I guess the question is how aggressively are we trying to re-hash. It would be fine for me either way. Please let me know if you would prefer it changed.

##########
File path: server/base/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java
##########
@@ -88,14 +90,40 @@ public void testTableConversion() {
 
   @Test
   public void testEncryption() {
+    byte[] rawPass = "myPassword".getBytes(Charset.forName("UTF-8"));
+    byte[] storedBytes;
+    try {
+      storedBytes = ZKSecurityTool.createPass(rawPass.clone());
+      assertTrue(ZKSecurityTool.checkCryptPass(rawPass.clone(), storedBytes));
+    } catch (AccumuloException e) {
+      log.error("{}", e.getMessage(), e);
+      fail();
+    }

Review comment:
       Thank you for the suggestion!

##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -116,6 +133,31 @@ public void createUser(String principal, AuthenticationToken token)
     }
   }
 
+  /**
+   * Creates user with outdated password hash for testing
+   *
+   * @deprecated since 2.1.0, only present for testing DO NOT USE!
+   */

Review comment:
       Why do you feel a warning is necessary? It's not a widely used method and the original name was re-used for the new functionality. But if we would add a safeguard I would prefer to throw an AccumuloSecurityException at the end of the method. We can catch it in test context and would be harder to miss for a dev than a log msg. What do you think?

##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthenticator.java
##########
@@ -52,6 +52,23 @@ public void initialize(ServerContext context) {
     this.context = context;
     zooCache = new ZooCache(context.getZooReaderWriter(), null);
     ZKUserPath = Constants.ZROOT + "/" + context.getInstanceID() + "/users";
+    checkOutdatedHashes();
+  }
+
+  private void checkOutdatedHashes() {
+    try {
+      listUsers().forEach(user -> {
+        String zpath = ZKUserPath + "/" + user;
+        byte[] zkData = zooCache.get(zpath);
+        if (ZKSecurityTool.isOutdatedPass(zkData)) {
+          log.warn("Found user(s) with outdated password hash. These will be re-hashed"
+              + " on successful authentication.");
+          return;
+        }
+      });
+    } catch (NullPointerException e) {
+      // initializeSecurity was not called yet, there could be no outdated passwords stored

Review comment:
       The zknode missing should be a corner case so I would prefer to keep the try/catch. I'll add a debug log message.

##########
File path: core/src/main/java/org/apache/accumulo/core/Constants.java
##########
@@ -99,7 +99,8 @@
   public static final long SCANNER_DEFAULT_READAHEAD_THRESHOLD = 3L;
 
   // Security configuration
-  public static final String PW_HASH_ALGORITHM = "SHA-256";
+  public static final String PW_HASH_ALGORITHM = "SHA-512";

Review comment:
       You are right. I'll make the system credentials hash customizable while defaulting to 256,  and renaming this as suggested to reflect it's for unsecure use.

##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java
##########
@@ -56,16 +59,26 @@
     return salt;
   }
 
+  // only present for testing DO NOT USE!
+  @Deprecated(since = "2.1.0")
+  static byte[] createOutdatedPass(byte[] password) throws AccumuloException {
+    byte[] salt = generateSalt();
+    try {
+      return convertPass(password, salt);
+    } catch (NoSuchAlgorithmException e) {
+      log.error("Count not create hashed password", e);
+      throw new AccumuloException("Count not create hashed password", e);
+    }
+  }
+
   private static byte[] hash(byte[] raw) throws NoSuchAlgorithmException {
-    MessageDigest md = MessageDigest.getInstance(Constants.PW_HASH_ALGORITHM);
+    MessageDigest md = MessageDigest.getInstance(Constants.PW_HASH_ALGORITHM_OUTDATED);
     md.update(raw);
     return md.digest();
   }
 
+  @Deprecated(since = "2.1.0")
   public static boolean checkPass(byte[] password, byte[] zkData) {
-    if (zkData == null)
-      return false;
-

Review comment:
       I re-added the check to be on the safe side, but and made it package private as I should have done already.

##########
File path: server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java
##########
@@ -99,6 +102,24 @@ public static boolean checkPass(byte[] password, byte[] zkData) {
     return saltedHash; // contains salt+hash(password+salt)
   }
 
+  public static byte[] createPass(byte[] password) throws AccumuloException {
+    // we rely on default algorithm and hash length (SHA-512 and 8 byte)
+    String cryptHash = Crypt.crypt(password);
+    return cryptHash.getBytes(CRYPT_CHARSET);
+  }
+
+  public static boolean checkCryptPass(byte[] password, byte[] zkData) {
+    String zkDataString = new String(zkData, CRYPT_CHARSET);
+    String cryptHash;
+    try {
+      cryptHash = Crypt.crypt(password, zkDataString);
+    } catch (IllegalArgumentException e) {
+      log.error("Unrecognized hash format", e);
+      return false;
+    }
+    return MessageDigest.isEqual(zkData, cryptHash.getBytes(CRYPT_CHARSET));

Review comment:
       String comparisons is not secure against timing attacks, `MessageDigest.isEqual` is.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org