You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by "dlmarion (via GitHub)" <gi...@apache.org> on 2023/05/30 18:06:32 UTC

[GitHub] [accumulo] dlmarion opened a new pull request, #3440: Modify Shell to authenticate user on call to config

dlmarion opened a new pull request, #3440:
URL: https://github.com/apache/accumulo/pull/3440

   (no comment)


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] keith-turner commented on a diff in pull request #3440: Modify Shell to authenticate user on call to config

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3440:
URL: https://github.com/apache/accumulo/pull/3440#discussion_r1211720032


##########
shell/src/main/java/org/apache/accumulo/shell/Shell.java:
##########
@@ -189,6 +189,16 @@
  */
 @AutoService(KeywordExecutable.class)
 public class Shell extends ShellOptions implements KeywordExecutable {
+
+  public static interface Authenticator {
+    default boolean authenticateUser(AccumuloClient client, String principal,
+        AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
+      return client.securityOperations().authenticateUser(principal, token);
+    }
+  }
+
+  public static Authenticator AUTHENTICATOR = new Authenticator() {};

Review Comment:
   Does this exist so that a test can override a static method?



##########
test/src/main/java/org/apache/accumulo/test/shell/ShellAuthenticatorTest.java:
##########
@@ -0,0 +1,121 @@
+package org.apache.accumulo.test.shell;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Properties;
+import java.util.TimeZone;
+
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.shell.Shell;
+import org.apache.accumulo.test.shell.ShellIT.StringInputStream;
+import org.apache.accumulo.test.shell.ShellIT.TestOutputStream;
+import org.jline.reader.LineReader;
+import org.jline.reader.LineReaderBuilder;
+import org.jline.terminal.Size;
+import org.jline.terminal.Terminal;
+import org.jline.terminal.impl.DumbTerminal;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+public class ShellAuthenticatorTest extends SharedMiniClusterBase {
+
+  @BeforeAll
+  public static void setup() throws Exception {
+    SharedMiniClusterBase.startMiniCluster();
+  }
+
+  @AfterAll
+  public static void teardown() {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  private StringInputStream input;
+  private TestOutputStream output;
+  private Shell shell;
+  private File config;
+  public LineReader reader;
+  public Terminal terminal;
+  
+  @BeforeEach
+  public void setupShell() throws IOException {
+    TimeZone.setDefault(TimeZone.getTimeZone("UTC"));
+    output = new TestOutputStream();
+    input = new StringInputStream();
+    terminal = new DumbTerminal(input, output);
+    terminal.setSize(new Size(80, 24));
+    reader = LineReaderBuilder.builder().terminal(terminal).build();
+    if (config != null) {
+      config.delete();
+    }
+    config = Files.createTempFile(null, null).toFile();
+    try (FileWriter writer = new FileWriter(config)) {
+      Properties p = super.getClientProps();
+      p.store(writer, null);
+    }
+    config.deleteOnExit();
+  }
+  
+  @AfterEach
+  public void tearDownShell() { 
+    if (shell != null) {
+      shell.shutdown();
+    }
+  }
+
+  @Test
+  public void testClientPropertiesFile() throws IOException {
+    shell = new Shell(reader);
+    shell.setLogErrorsToConsole();
+    assertTrue(shell.config("--config-file", config.toString()));
+  }
+
+  @Test
+  public void testClientProperties() throws IOException {
+    shell = new Shell(reader);
+    shell.setLogErrorsToConsole();
+    assertTrue(shell.config("-u", "root", "-p", getRootPassword(), "-zi",
+        getCluster().getInstanceName(), "-zh", getCluster().getZooKeepers()));
+  }
+
+  @Test
+  public void testClientPropertiesBadPassword() throws IOException {

Review Comment:
   How does it fail when run with other test?  What errors to do you see?



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] dlmarion commented on a diff in pull request #3440: Modify Shell to authenticate user on call to config

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on code in PR #3440:
URL: https://github.com/apache/accumulo/pull/3440#discussion_r1210642309


##########
test/src/main/java/org/apache/accumulo/test/shell/ShellAuthenticatorTest.java:
##########
@@ -0,0 +1,121 @@
+package org.apache.accumulo.test.shell;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Properties;
+import java.util.TimeZone;
+
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.shell.Shell;
+import org.apache.accumulo.test.shell.ShellIT.StringInputStream;
+import org.apache.accumulo.test.shell.ShellIT.TestOutputStream;
+import org.jline.reader.LineReader;
+import org.jline.reader.LineReaderBuilder;
+import org.jline.terminal.Size;
+import org.jline.terminal.Terminal;
+import org.jline.terminal.impl.DumbTerminal;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+public class ShellAuthenticatorTest extends SharedMiniClusterBase {
+
+  @BeforeAll
+  public static void setup() throws Exception {
+    SharedMiniClusterBase.startMiniCluster();
+  }
+
+  @AfterAll
+  public static void teardown() {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  private StringInputStream input;
+  private TestOutputStream output;
+  private Shell shell;
+  private File config;
+  public LineReader reader;
+  public Terminal terminal;
+  
+  @BeforeEach
+  public void setupShell() throws IOException {
+    TimeZone.setDefault(TimeZone.getTimeZone("UTC"));
+    output = new TestOutputStream();
+    input = new StringInputStream();
+    terminal = new DumbTerminal(input, output);
+    terminal.setSize(new Size(80, 24));
+    reader = LineReaderBuilder.builder().terminal(terminal).build();
+    if (config != null) {
+      config.delete();
+    }
+    config = Files.createTempFile(null, null).toFile();
+    try (FileWriter writer = new FileWriter(config)) {
+      Properties p = super.getClientProps();
+      p.store(writer, null);
+    }
+    config.deleteOnExit();
+  }
+  
+  @AfterEach
+  public void tearDownShell() { 
+    if (shell != null) {
+      shell.shutdown();
+    }
+  }
+
+  @Test
+  public void testClientPropertiesFile() throws IOException {
+    shell = new Shell(reader);
+    shell.setLogErrorsToConsole();
+    assertTrue(shell.config("--config-file", config.toString()));
+  }
+
+  @Test
+  public void testClientProperties() throws IOException {
+    shell = new Shell(reader);
+    shell.setLogErrorsToConsole();
+    assertTrue(shell.config("-u", "root", "-p", getRootPassword(), "-zi",
+        getCluster().getInstanceName(), "-zh", getCluster().getZooKeepers()));
+  }
+
+  @Test
+  public void testClientPropertiesBadPassword() throws IOException {

Review Comment:
   This test is failing for me intermittently and I'm not sure why. If I run the test by itself or in the debugger it works. If I run it with the other tests in this file, then it fails.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #3440: Modify Shell to authenticate user on call to config

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on code in PR #3440:
URL: https://github.com/apache/accumulo/pull/3440#discussion_r1223305274


##########
shell/src/main/java/org/apache/accumulo/shell/Shell.java:
##########
@@ -189,6 +189,16 @@
  */
 @AutoService(KeywordExecutable.class)
 public class Shell extends ShellOptions implements KeywordExecutable {
+
+  public static interface Authenticator {
+    default boolean authenticateUser(AccumuloClient client, String principal,
+        AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
+      return client.securityOperations().authenticateUser(principal, token);
+    }
+  }
+
+  public static Authenticator AUTHENTICATOR = new Authenticator() {};

Review Comment:
   Note: FateCommandTest is removed in 3.0, since the FateCommand is deprecated in 2.1 and removed in 3.0.
   
   I think that making this static is a confusing way to implement this for testing, though. I think it would be better to use a protected method and create a subclass test shell in FateCommandTest.createShell that overrides this method for testing. That way, there's less public and less static stuff leaking out of the shell's API for users to shoot themselves in the foot with.
   
   I can add a commit to this PR or submit a PR to your branch to demonstrate my suggestion, if you want. Whichever you'd prefer.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] EdColeman commented on a diff in pull request #3440: Modify Shell to authenticate user on call to config

Posted by "EdColeman (via GitHub)" <gi...@apache.org>.
EdColeman commented on code in PR #3440:
URL: https://github.com/apache/accumulo/pull/3440#discussion_r1222222342


##########
test/src/main/java/org/apache/accumulo/test/shell/ShellAuthenticatorIT.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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
+ *
+ *   https://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.shell;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Properties;
+import java.util.TimeZone;
+
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.shell.Shell;
+import org.apache.accumulo.test.shell.ShellIT.StringInputStream;
+import org.apache.accumulo.test.shell.ShellIT.TestOutputStream;
+import org.jline.reader.LineReader;
+import org.jline.reader.LineReaderBuilder;
+import org.jline.terminal.Size;
+import org.jline.terminal.Terminal;
+import org.jline.terminal.impl.DumbTerminal;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+public class ShellAuthenticatorIT extends SharedMiniClusterBase {
+
+  @BeforeAll
+  public static void setup() throws Exception {
+    SharedMiniClusterBase.startMiniCluster();
+  }
+
+  @AfterAll
+  public static void teardown() {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  private StringInputStream input;
+  private TestOutputStream output;
+  private Shell shell;
+  private File config;
+  public LineReader reader;
+  public Terminal terminal;
+
+  @BeforeEach
+  public void setupShell() throws IOException {
+    TimeZone.setDefault(TimeZone.getTimeZone("UTC"));
+    output = new TestOutputStream();
+    input = new StringInputStream();
+    terminal = new DumbTerminal(input, output);
+    terminal.setSize(new Size(80, 24));
+    reader = LineReaderBuilder.builder().terminal(terminal).build();
+    if (config != null) {
+      config.delete();

Review Comment:
   ```suggestion
         if(!config.delete()){
           throw new IllegalStateException("failed to delete: " + config.getAbsolutePath());
         }
   ```



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] dlmarion merged pull request #3440: Modify Shell to authenticate user on call to config

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion merged PR #3440:
URL: https://github.com/apache/accumulo/pull/3440


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] dlmarion commented on pull request #3440: Modify Shell to authenticate user on call to config

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on PR #3440:
URL: https://github.com/apache/accumulo/pull/3440#issuecomment-1583170466

   No issue 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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] EdColeman commented on a diff in pull request #3440: Modify Shell to authenticate user on call to config

Posted by "EdColeman (via GitHub)" <gi...@apache.org>.
EdColeman commented on code in PR #3440:
URL: https://github.com/apache/accumulo/pull/3440#discussion_r1222221840


##########
test/src/main/java/org/apache/accumulo/test/shell/ShellAuthenticatorIT.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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
+ *
+ *   https://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.shell;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Properties;
+import java.util.TimeZone;
+
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.shell.Shell;
+import org.apache.accumulo.test.shell.ShellIT.StringInputStream;
+import org.apache.accumulo.test.shell.ShellIT.TestOutputStream;
+import org.jline.reader.LineReader;
+import org.jline.reader.LineReaderBuilder;
+import org.jline.terminal.Size;
+import org.jline.terminal.Terminal;
+import org.jline.terminal.impl.DumbTerminal;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+public class ShellAuthenticatorIT extends SharedMiniClusterBase {
+
+  @BeforeAll
+  public static void setup() throws Exception {
+    SharedMiniClusterBase.startMiniCluster();
+  }
+
+  @AfterAll
+  public static void teardown() {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  private StringInputStream input;
+  private TestOutputStream output;
+  private Shell shell;
+  private File config;
+  public LineReader reader;
+  public Terminal terminal;
+
+  @BeforeEach
+  public void setupShell() throws IOException {
+    TimeZone.setDefault(TimeZone.getTimeZone("UTC"));
+    output = new TestOutputStream();
+    input = new StringInputStream();
+    terminal = new DumbTerminal(input, output);
+    terminal.setSize(new Size(80, 24));
+    reader = LineReaderBuilder.builder().terminal(terminal).build();
+    if (config != null) {
+      config.delete();
+    }
+    config = Files.createTempFile(null, null).toFile();
+    try (FileWriter writer = new FileWriter(config, UTF_8)) {
+      Properties p = SharedMiniClusterBase.getClientProps();
+      p.store(writer, null);
+    }
+    config.deleteOnExit();
+  }
+
+  @AfterEach
+  public void tearDownShell() {
+    if (shell != null) {
+      shell.shutdown();
+    }
+  }
+
+  @Test
+  public void testClientPropertiesFile() throws IOException {
+    shell = new Shell(reader);
+    shell.setLogErrorsToConsole();
+    assertTrue(shell.config("--config-file", config.toString()));
+  }
+
+  @Test
+  public void testClientProperties() throws IOException {
+    shell = new Shell(reader);
+    shell.setLogErrorsToConsole();
+    assertTrue(shell.config("-u", getAdminPrincipal(), "-p", getRootPassword(), "-zi",
+        getCluster().getInstanceName(), "-zh", getCluster().getZooKeepers()));
+  }
+
+  @Test
+  public void testClientPropertiesBadPassword() throws IOException {
+    shell = new Shell(reader);
+    shell.setLogErrorsToConsole();
+    assertFalse(shell.config("-u", getAdminPrincipal(), "-p", "BADPW", "-zi",
+        getCluster().getInstanceName(), "-zh", getCluster().getZooKeepers()));
+  }
+
+  @Test
+  public void testAuthTimeoutPropertiesFile() throws IOException, InterruptedException {
+    TimeZone.setDefault(TimeZone.getTimeZone("UTC"));
+    output = new TestOutputStream();
+    input = new StringInputStream();
+    terminal = new DumbTerminal(input, output);
+    terminal.setSize(new Size(80, 24));
+    reader = LineReaderBuilder.builder().terminal(terminal).build();
+    if (config != null) {
+      config.delete();

Review Comment:
   ```suggestion
        if(!config.delete()){
           throw new IllegalStateException("failed to delete: " + config.getAbsolutePath());
         }
   ```



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #3440: Modify Shell to authenticate user on call to config

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on code in PR #3440:
URL: https://github.com/apache/accumulo/pull/3440#discussion_r1223305274


##########
shell/src/main/java/org/apache/accumulo/shell/Shell.java:
##########
@@ -189,6 +189,16 @@
  */
 @AutoService(KeywordExecutable.class)
 public class Shell extends ShellOptions implements KeywordExecutable {
+
+  public static interface Authenticator {
+    default boolean authenticateUser(AccumuloClient client, String principal,
+        AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
+      return client.securityOperations().authenticateUser(principal, token);
+    }
+  }
+
+  public static Authenticator AUTHENTICATOR = new Authenticator() {};

Review Comment:
   Note: FateCommandTest is removed in 3.0, since the FateCommand is deprecated in 2.1 and removed in 3.0.
   
   I think that making this static is a confusing way to implement this for testing, though. I think it would be better to use a protected method and create a subclass test shell in FateCommandTest.createShell that overrides this method for testing. That way, there's less public and less static stuff leaking out of the shell's API for users to shoot themselves in the foot with.
   
   I can add a commit or submit a PR to your branch to demonstrate my suggestion, if you want. Whichever you'd prefer.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #3440: Modify Shell to authenticate user on call to config

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on code in PR #3440:
URL: https://github.com/apache/accumulo/pull/3440#discussion_r1223305274


##########
shell/src/main/java/org/apache/accumulo/shell/Shell.java:
##########
@@ -189,6 +189,16 @@
  */
 @AutoService(KeywordExecutable.class)
 public class Shell extends ShellOptions implements KeywordExecutable {
+
+  public static interface Authenticator {
+    default boolean authenticateUser(AccumuloClient client, String principal,
+        AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
+      return client.securityOperations().authenticateUser(principal, token);
+    }
+  }
+
+  public static Authenticator AUTHENTICATOR = new Authenticator() {};

Review Comment:
   Note: FateCommandTest is removed in 3.0, since the FateCommand is deprecated in 2.1 and removed in 3.0.
   
   I think that making this static is a confusing way to implement this for testing, though. I think it would be better to use a protected method and create a subclass test shell in FateCommandTest.createShell that overrides this method for testing. That way, there's less public and less static stuff leaking out of the shell's API for users to shoot themselves in the foot with.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] dlmarion commented on pull request #3440: Modify Shell to authenticate user on call to config

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on PR #3440:
URL: https://github.com/apache/accumulo/pull/3440#issuecomment-1581540012

   Findbugs is complaining about:
   
   ```
   Medium: Hard coded password found [org.apache.accumulo.shell.Shell, org.apache.accumulo.shell.Shell, org.apache.accumulo.shell.Shell, org.apache.accumulo.shell.ShellOptionsJC] At Shell.java:[line 283]At Shell.java:[line 370]At Shell.java:[line 371]At ShellOptionsJC.java:[line 148] HARD_CODE_PASSWORD
   ```
   
   Not sure what the issue is, there is no hardcoded password


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] keith-turner commented on a diff in pull request #3440: Modify Shell to authenticate user on call to config

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3440:
URL: https://github.com/apache/accumulo/pull/3440#discussion_r1211797793


##########
shell/src/main/java/org/apache/accumulo/shell/Shell.java:
##########
@@ -189,6 +189,16 @@
  */
 @AutoService(KeywordExecutable.class)
 public class Shell extends ShellOptions implements KeywordExecutable {
+
+  public static interface Authenticator {
+    default boolean authenticateUser(AccumuloClient client, String principal,
+        AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
+      return client.securityOperations().authenticateUser(principal, token);
+    }
+  }
+
+  public static Authenticator AUTHENTICATOR = new Authenticator() {};

Review Comment:
   ok after looking at the PR that is what I suspected.  When I first saw `AUTHENTICATOR` I was wondering why it could not be replaced with a static method. 



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] dlmarion commented on a diff in pull request #3440: Modify Shell to authenticate user on call to config

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on code in PR #3440:
URL: https://github.com/apache/accumulo/pull/3440#discussion_r1211834671


##########
test/src/main/java/org/apache/accumulo/test/shell/ShellAuthenticatorTest.java:
##########
@@ -0,0 +1,121 @@
+package org.apache.accumulo.test.shell;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Properties;
+import java.util.TimeZone;
+
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.shell.Shell;
+import org.apache.accumulo.test.shell.ShellIT.StringInputStream;
+import org.apache.accumulo.test.shell.ShellIT.TestOutputStream;
+import org.jline.reader.LineReader;
+import org.jline.reader.LineReaderBuilder;
+import org.jline.terminal.Size;
+import org.jline.terminal.Terminal;
+import org.jline.terminal.impl.DumbTerminal;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+public class ShellAuthenticatorTest extends SharedMiniClusterBase {
+
+  @BeforeAll
+  public static void setup() throws Exception {
+    SharedMiniClusterBase.startMiniCluster();
+  }
+
+  @AfterAll
+  public static void teardown() {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  private StringInputStream input;
+  private TestOutputStream output;
+  private Shell shell;
+  private File config;
+  public LineReader reader;
+  public Terminal terminal;
+  
+  @BeforeEach
+  public void setupShell() throws IOException {
+    TimeZone.setDefault(TimeZone.getTimeZone("UTC"));
+    output = new TestOutputStream();
+    input = new StringInputStream();
+    terminal = new DumbTerminal(input, output);
+    terminal.setSize(new Size(80, 24));
+    reader = LineReaderBuilder.builder().terminal(terminal).build();
+    if (config != null) {
+      config.delete();
+    }
+    config = Files.createTempFile(null, null).toFile();
+    try (FileWriter writer = new FileWriter(config)) {
+      Properties p = super.getClientProps();
+      p.store(writer, null);
+    }
+    config.deleteOnExit();
+  }
+  
+  @AfterEach
+  public void tearDownShell() { 
+    if (shell != null) {
+      shell.shutdown();
+    }
+  }
+
+  @Test
+  public void testClientPropertiesFile() throws IOException {
+    shell = new Shell(reader);
+    shell.setLogErrorsToConsole();
+    assertTrue(shell.config("--config-file", config.toString()));
+  }
+
+  @Test
+  public void testClientProperties() throws IOException {
+    shell = new Shell(reader);
+    shell.setLogErrorsToConsole();
+    assertTrue(shell.config("-u", "root", "-p", getRootPassword(), "-zi",
+        getCluster().getInstanceName(), "-zh", getCluster().getZooKeepers()));
+  }
+
+  @Test
+  public void testClientPropertiesBadPassword() throws IOException {

Review Comment:
   No error server-side, in IDE it's:
   
   ```
   org.opentest4j.AssertionFailedError: expected: <false> but was: <true>
   	at org.junit.jupiter.api.AssertionFailureBuilder.build(AssertionFailureBuilder.java:151)
   	at org.junit.jupiter.api.AssertionFailureBuilder.buildAndThrow(AssertionFailureBuilder.java:132)
   	at org.junit.jupiter.api.AssertFalse.failNotFalse(AssertFalse.java:63)
   	at org.junit.jupiter.api.AssertFalse.assertFalse(AssertFalse.java:36)
   	at org.junit.jupiter.api.AssertFalse.assertFalse(AssertFalse.java:31)
   	at org.junit.jupiter.api.Assertions.assertFalse(Assertions.java:228)
   	at org.apache.accumulo.test.shell.ShellAuthenticatorIT.testClientPropertiesBadPassword(ShellAuthenticatorIT.java:110)
   ...
   ```



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] cshannon commented on pull request #3440: Modify Shell to authenticate user on call to config

Posted by "cshannon (via GitHub)" <gi...@apache.org>.
cshannon commented on PR #3440:
URL: https://github.com/apache/accumulo/pull/3440#issuecomment-1570512732

   The failure appears to be related to the CRYPT_PASSWORD_CACHE inside of ZKSecurityTool. I commented out this line and the test no longer failures: https://github.com/apache/accumulo/blob/969fc52f1c5e0c195a8cef800a87e0fab13cd89c/server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKSecurityTool.java#L152


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] dlmarion commented on a diff in pull request #3440: Modify Shell to authenticate user on call to config

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on code in PR #3440:
URL: https://github.com/apache/accumulo/pull/3440#discussion_r1211788542


##########
shell/src/main/java/org/apache/accumulo/shell/Shell.java:
##########
@@ -189,6 +189,16 @@
  */
 @AutoService(KeywordExecutable.class)
 public class Shell extends ShellOptions implements KeywordExecutable {
+
+  public static interface Authenticator {
+    default boolean authenticateUser(AccumuloClient client, String principal,
+        AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
+      return client.securityOperations().authenticateUser(principal, token);
+    }
+  }
+
+  public static Authenticator AUTHENTICATOR = new Authenticator() {};

Review Comment:
   Yes, FateCommandTest



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #3440: Modify Shell to authenticate user on call to config

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on code in PR #3440:
URL: https://github.com/apache/accumulo/pull/3440#discussion_r1223431112


##########
shell/src/main/java/org/apache/accumulo/shell/Shell.java:
##########
@@ -189,6 +189,16 @@
  */
 @AutoService(KeywordExecutable.class)
 public class Shell extends ShellOptions implements KeywordExecutable {
+
+  public static interface Authenticator {
+    default boolean authenticateUser(AccumuloClient client, String principal,
+        AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
+      return client.securityOperations().authenticateUser(principal, token);
+    }
+  }
+
+  public static Authenticator AUTHENTICATOR = new Authenticator() {};

Review Comment:
   I added commit 2db232fc79eaf640d43c81da4fba19dab7bcd0e7 to implement my suggestions. This commit can be rolled back out or tweaked, if needed.



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] dlmarion commented on pull request #3440: Modify Shell to authenticate user on call to config

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on PR #3440:
URL: https://github.com/apache/accumulo/pull/3440#issuecomment-1583187210

   @ctubbsii - It looks like you reintroduced the sec-bugs issue:
   ```
   Medium: Hard coded password found [org.apache.accumulo.shell.Shell, org.apache.accumulo.shell.Shell, org.apache.accumulo.shell.Shell, org.apache.accumulo.shell.ShellOptionsJC] At Shell.java:[line 276]At Shell.java:[line 363]At Shell.java:[line 364]At ShellOptionsJC.java:[line 148] HARD_CODE_PASSWORD
   ```
   
   Some of the sec-bugs password [plugins](https://github.com/find-sec-bugs/find-sec-bugs/tree/master/findsecbugs-plugin/src/main/java/com/h3xstream/findsecbugs/password) contain patterns for variable names. That's why I chose to use a different variable name.


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] dlmarion commented on a diff in pull request #3440: Modify Shell to authenticate user on call to config

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on code in PR #3440:
URL: https://github.com/apache/accumulo/pull/3440#discussion_r1222929424


##########
test/src/main/java/org/apache/accumulo/test/shell/ShellAuthenticatorIT.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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
+ *
+ *   https://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.shell;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Properties;
+import java.util.TimeZone;
+
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.shell.Shell;
+import org.apache.accumulo.test.shell.ShellIT.StringInputStream;
+import org.apache.accumulo.test.shell.ShellIT.TestOutputStream;
+import org.jline.reader.LineReader;
+import org.jline.reader.LineReaderBuilder;
+import org.jline.terminal.Size;
+import org.jline.terminal.Terminal;
+import org.jline.terminal.impl.DumbTerminal;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+public class ShellAuthenticatorIT extends SharedMiniClusterBase {
+
+  @BeforeAll
+  public static void setup() throws Exception {
+    SharedMiniClusterBase.startMiniCluster();
+  }
+
+  @AfterAll
+  public static void teardown() {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  private StringInputStream input;
+  private TestOutputStream output;
+  private Shell shell;
+  private File config;
+  public LineReader reader;
+  public Terminal terminal;
+
+  @BeforeEach
+  public void setupShell() throws IOException {
+    TimeZone.setDefault(TimeZone.getTimeZone("UTC"));
+    output = new TestOutputStream();
+    input = new StringInputStream();
+    terminal = new DumbTerminal(input, output);
+    terminal.setSize(new Size(80, 24));
+    reader = LineReaderBuilder.builder().terminal(terminal).build();
+    if (config != null) {
+      config.delete();
+    }
+    config = Files.createTempFile(null, null).toFile();
+    try (FileWriter writer = new FileWriter(config, UTF_8)) {
+      Properties p = SharedMiniClusterBase.getClientProps();
+      p.store(writer, null);
+    }
+    config.deleteOnExit();
+  }
+
+  @AfterEach
+  public void tearDownShell() {
+    if (shell != null) {
+      shell.shutdown();
+    }
+  }
+
+  @Test
+  public void testClientPropertiesFile() throws IOException {
+    shell = new Shell(reader);
+    shell.setLogErrorsToConsole();
+    assertTrue(shell.config("--config-file", config.toString()));
+  }
+
+  @Test
+  public void testClientProperties() throws IOException {
+    shell = new Shell(reader);
+    shell.setLogErrorsToConsole();
+    assertTrue(shell.config("-u", getAdminPrincipal(), "-p", getRootPassword(), "-zi",
+        getCluster().getInstanceName(), "-zh", getCluster().getZooKeepers()));
+  }
+
+  @Test
+  public void testClientPropertiesBadPassword() throws IOException {
+    shell = new Shell(reader);
+    shell.setLogErrorsToConsole();
+    assertFalse(shell.config("-u", getAdminPrincipal(), "-p", "BADPW", "-zi",
+        getCluster().getInstanceName(), "-zh", getCluster().getZooKeepers()));
+  }
+
+  @Test
+  public void testAuthTimeoutPropertiesFile() throws IOException, InterruptedException {
+    TimeZone.setDefault(TimeZone.getTimeZone("UTC"));
+    output = new TestOutputStream();
+    input = new StringInputStream();
+    terminal = new DumbTerminal(input, output);
+    terminal.setSize(new Size(80, 24));
+    reader = LineReaderBuilder.builder().terminal(terminal).build();
+    if (config != null) {
+      config.delete();

Review Comment:
   Added in 8f71e5c



##########
test/src/main/java/org/apache/accumulo/test/shell/ShellAuthenticatorIT.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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
+ *
+ *   https://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.shell;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Properties;
+import java.util.TimeZone;
+
+import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.shell.Shell;
+import org.apache.accumulo.test.shell.ShellIT.StringInputStream;
+import org.apache.accumulo.test.shell.ShellIT.TestOutputStream;
+import org.jline.reader.LineReader;
+import org.jline.reader.LineReaderBuilder;
+import org.jline.terminal.Size;
+import org.jline.terminal.Terminal;
+import org.jline.terminal.impl.DumbTerminal;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+public class ShellAuthenticatorIT extends SharedMiniClusterBase {
+
+  @BeforeAll
+  public static void setup() throws Exception {
+    SharedMiniClusterBase.startMiniCluster();
+  }
+
+  @AfterAll
+  public static void teardown() {
+    SharedMiniClusterBase.stopMiniCluster();
+  }
+
+  private StringInputStream input;
+  private TestOutputStream output;
+  private Shell shell;
+  private File config;
+  public LineReader reader;
+  public Terminal terminal;
+
+  @BeforeEach
+  public void setupShell() throws IOException {
+    TimeZone.setDefault(TimeZone.getTimeZone("UTC"));
+    output = new TestOutputStream();
+    input = new StringInputStream();
+    terminal = new DumbTerminal(input, output);
+    terminal.setSize(new Size(80, 24));
+    reader = LineReaderBuilder.builder().terminal(terminal).build();
+    if (config != null) {
+      config.delete();

Review Comment:
   Added in 8f71e5c



-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] dlmarion commented on pull request #3440: Modify Shell to authenticate user on call to config

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on PR #3440:
URL: https://github.com/apache/accumulo/pull/3440#issuecomment-1582471539

   It looks like changing the variable name from `password` to something that the findbugs password checking plugins are not looking for enables the build to pass. My guess is that it thought that the password was hardcoded in ShellOptionsJC and then highlighted everywhere it was referenced.


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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


[GitHub] [accumulo] ctubbsii commented on pull request #3440: Modify Shell to authenticate user on call to config

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on PR #3440:
URL: https://github.com/apache/accumulo/pull/3440#issuecomment-1583197353

   > @ctubbsii - It looks like you reintroduced the sec-bugs issue:
   
   Yep, sorry about that. I missed that one when I was resolving conflicts with my code review commit. Should be fixed now.


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

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