You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2022/08/11 13:14:38 UTC

[GitHub] [ignite-3] valepakh opened a new pull request, #1001: IGNITE-17469 cli profile commands unification

valepakh opened a new pull request, #1001:
URL: https://github.com/apache/ignite-3/pull/1001

   https://issues.apache.org/jira/browse/IGNITE-17469
   
   This unifies the `cli config profile` commands to use parameters rather than options for profile name and uses profile options for all node/cluster commands which use `--node-url` or `--cluster-endpoint-url` to take the default values from the profile specified by `--profile` option.


-- 
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@ignite.apache.org

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


[GitHub] [ignite-3] valepakh commented on a diff in pull request #1001: IGNITE-17469 cli profile commands unification

Posted by GitBox <gi...@apache.org>.
valepakh commented on code in PR #1001:
URL: https://github.com/apache/ignite-3/pull/1001#discussion_r947896393


##########
modules/cli/src/main/java/org/apache/ignite/cli/call/cliconfig/profile/CliConfigProfileListCall.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.ignite.cli.call.cliconfig.profile;
+
+import jakarta.inject.Inject;
+import jakarta.inject.Singleton;
+import java.util.Collection;
+import java.util.stream.Collectors;
+import org.apache.ignite.cli.config.ConfigManagerProvider;
+import org.apache.ignite.cli.core.call.Call;
+import org.apache.ignite.cli.core.call.CallOutput;
+import org.apache.ignite.cli.core.call.DefaultCallOutput;
+import org.apache.ignite.cli.core.call.EmptyCallInput;
+
+/**
+ * List profiles call.
+ */
+@Singleton
+public class CliConfigProfileListCall implements Call<EmptyCallInput, String> {
+
+    @Inject
+    private ConfigManagerProvider provider;
+
+    @Override
+    public CallOutput<String> execute(EmptyCallInput input) {
+        Collection<String> profileNames = provider.get().getProfileNames();
+        String result = profileNames.stream().collect(Collectors.joining(System.lineSeparator()));

Review Comment:
   I added the mark for current profile in case output is not redirected.



-- 
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@ignite.apache.org

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


[GitHub] [ignite-3] YuryYudin commented on a diff in pull request #1001: IGNITE-17469 cli profile commands unification

Posted by GitBox <gi...@apache.org>.
YuryYudin commented on code in PR #1001:
URL: https://github.com/apache/ignite-3/pull/1001#discussion_r947757213


##########
modules/cli/src/main/java/org/apache/ignite/cli/call/cliconfig/profile/CliConfigProfileListCall.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.ignite.cli.call.cliconfig.profile;
+
+import jakarta.inject.Inject;
+import jakarta.inject.Singleton;
+import java.util.Collection;
+import java.util.stream.Collectors;
+import org.apache.ignite.cli.config.ConfigManagerProvider;
+import org.apache.ignite.cli.core.call.Call;
+import org.apache.ignite.cli.core.call.CallOutput;
+import org.apache.ignite.cli.core.call.DefaultCallOutput;
+import org.apache.ignite.cli.core.call.EmptyCallInput;
+
+/**
+ * List profiles call.
+ */
+@Singleton
+public class CliConfigProfileListCall implements Call<EmptyCallInput, String> {
+
+    @Inject
+    private ConfigManagerProvider provider;
+
+    @Override
+    public CallOutput<String> execute(EmptyCallInput input) {
+        Collection<String> profileNames = provider.get().getProfileNames();
+        String result = profileNames.stream().collect(Collectors.joining(System.lineSeparator()));

Review Comment:
   We're not supposed to pipe  anything in the REPL mode. It would make sense to still mark it there. 



-- 
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@ignite.apache.org

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


[GitHub] [ignite-3] valepakh commented on a diff in pull request #1001: IGNITE-17469 cli profile commands unification

Posted by GitBox <gi...@apache.org>.
valepakh commented on code in PR #1001:
URL: https://github.com/apache/ignite-3/pull/1001#discussion_r945551072


##########
modules/cli/src/main/java/org/apache/ignite/cli/call/cliconfig/profile/CliConfigProfileListCall.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.ignite.cli.call.cliconfig.profile;
+
+import jakarta.inject.Inject;
+import jakarta.inject.Singleton;
+import java.util.Collection;
+import java.util.stream.Collectors;
+import org.apache.ignite.cli.config.ConfigManagerProvider;
+import org.apache.ignite.cli.core.call.Call;
+import org.apache.ignite.cli.core.call.CallOutput;
+import org.apache.ignite.cli.core.call.DefaultCallOutput;
+import org.apache.ignite.cli.core.call.EmptyCallInput;
+
+/**
+ * List profiles call.
+ */
+@Singleton
+public class CliConfigProfileListCall implements Call<EmptyCallInput, String> {
+
+    @Inject
+    private ConfigManagerProvider provider;
+
+    @Override
+    public CallOutput<String> execute(EmptyCallInput input) {
+        Collection<String> profileNames = provider.get().getProfileNames();
+        String result = profileNames.stream().collect(Collectors.joining(System.lineSeparator()));

Review Comment:
   I decided to not mark the active profile when I found an interesting use-case - pipe the output of this command to change some option in all profiles. We might want to add the mark only when the output is not piped though.



-- 
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@ignite.apache.org

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


[GitHub] [ignite-3] SammyVimes merged pull request #1001: IGNITE-17469 cli profile commands unification

Posted by GitBox <gi...@apache.org>.
SammyVimes merged PR #1001:
URL: https://github.com/apache/ignite-3/pull/1001


-- 
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@ignite.apache.org

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


[GitHub] [ignite-3] PakhomovAlexander commented on a diff in pull request #1001: IGNITE-17469 cli profile commands unification

Posted by GitBox <gi...@apache.org>.
PakhomovAlexander commented on code in PR #1001:
URL: https://github.com/apache/ignite-3/pull/1001#discussion_r944820904


##########
modules/cli/src/main/java/org/apache/ignite/cli/call/cliconfig/profile/CliConfigProfileCreateCallInput.java:
##########
@@ -50,35 +53,4 @@ public String getCopyFrom() {
     public boolean isActivate() {
         return activate;
     }
-
-    /**
-     * Builder of {@link CliConfigCreateProfileCallInput}.
-     */
-    public static class CliConfigCreateProfileCallInputBuilder {

Review Comment:
   I am quite confused with the fact that you removed the builder only for this input? Why dont remove all builders for CallInput? I think it is ok no to have a builder for the input that has only one field but if the class has more that one field it is quite useful to hava a builder. Bulder helps not to confuse parameter names.



-- 
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@ignite.apache.org

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


[GitHub] [ignite-3] PakhomovAlexander commented on a diff in pull request #1001: IGNITE-17469 cli profile commands unification

Posted by GitBox <gi...@apache.org>.
PakhomovAlexander commented on code in PR #1001:
URL: https://github.com/apache/ignite-3/pull/1001#discussion_r948392012


##########
modules/cli/src/main/java/org/apache/ignite/cli/commands/decorators/ProfileListDecorator.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.ignite.cli.commands.decorators;
+
+import static org.apache.ignite.cli.core.style.AnsiStringSupport.Style;
+import static org.apache.ignite.cli.core.style.AnsiStringSupport.ansi;
+
+import java.util.stream.Collectors;
+import org.apache.ignite.cli.call.cliconfig.profile.ProfileList;
+import org.apache.ignite.cli.core.decorator.Decorator;
+import org.apache.ignite.cli.core.decorator.TerminalOutput;
+
+/**
+ * Decorator for printing {@link ProfileList}.
+ */
+public class ProfileListDecorator implements Decorator<ProfileList, TerminalOutput> {
+    @Override
+    public TerminalOutput decorate(ProfileList data) {
+        if (isatty()) {
+            return decorateCurrentProfileName(data);
+        } else {
+            return () -> data.getProfileNames().stream().collect(Collectors.joining(System.lineSeparator()));
+        }
+    }
+
+    private TerminalOutput decorateCurrentProfileName(ProfileList data) {
+        String currentProfileName = data.getCurrentProfileName();
+        return () -> data.getProfileNames().stream()
+                .map(p -> {
+                    if (p.equals(currentProfileName)) {
+                        return ansi(Style.BOLD.mark("* " + p));

Review Comment:
   In case we merge the UI Components it is better to use them here.



##########
modules/cli/src/test/java/org/apache/ignite/cli/commands/cliconfig/CliConfigProfileListCommandTest.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.ignite.cli.commands.cliconfig;
+
+import static org.junit.jupiter.api.Assertions.assertAll;
+
+import org.apache.ignite.cli.commands.cliconfig.profile.CliConfigProfileListCommand;
+import org.apache.ignite.cli.config.ini.IniConfigManager;
+import org.junit.jupiter.api.Test;
+
+class CliConfigProfileListCommandTest extends CliConfigCommandTestBase {
+    @Override
+    protected Class<?> getCommandClass() {
+        return CliConfigProfileListCommand.class;
+    }
+
+    @Test
+    public void testWithDefaultProfile() {
+        execute();
+
+        String expectedResult = "owner" + System.lineSeparator()
+                + "database" + System.lineSeparator();
+        assertAll(
+                () -> assertOutputIs(expectedResult),
+                this::assertErrOutputIsEmpty
+        );
+    }
+
+    @Test
+    public void testSingleProfile() {
+        configManagerProvider.configManager = new IniConfigManager(TestConfigManagerHelper.createClusterUrlNonDefault());
+        execute();
+
+        String expectedResult = "default" + System.lineSeparator();

Review Comment:
   Could you rename `default` to `nonDefault`?



##########
modules/cli/src/test/java/org/apache/ignite/cli/commands/cliconfig/CliConfigProfileListCommandTest.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.ignite.cli.commands.cliconfig;
+
+import static org.junit.jupiter.api.Assertions.assertAll;
+
+import org.apache.ignite.cli.commands.cliconfig.profile.CliConfigProfileListCommand;
+import org.apache.ignite.cli.config.ini.IniConfigManager;
+import org.junit.jupiter.api.Test;
+
+class CliConfigProfileListCommandTest extends CliConfigCommandTestBase {
+    @Override
+    protected Class<?> getCommandClass() {
+        return CliConfigProfileListCommand.class;
+    }
+
+    @Test
+    public void testWithDefaultProfile() {
+        execute();
+
+        String expectedResult = "owner" + System.lineSeparator()
+                + "database" + System.lineSeparator();
+        assertAll(
+                () -> assertOutputIs(expectedResult),
+                this::assertErrOutputIsEmpty
+        );
+    }
+
+    @Test
+    public void testSingleProfile() {
+        configManagerProvider.configManager = new IniConfigManager(TestConfigManagerHelper.createClusterUrlNonDefault());
+        execute();
+
+        String expectedResult = "default" + System.lineSeparator();

Review Comment:
   As far as I can see we set up non default profile but check the default one.



-- 
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@ignite.apache.org

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


[GitHub] [ignite-3] valepakh commented on a diff in pull request #1001: IGNITE-17469 cli profile commands unification

Posted by GitBox <gi...@apache.org>.
valepakh commented on code in PR #1001:
URL: https://github.com/apache/ignite-3/pull/1001#discussion_r948684530


##########
modules/cli/src/test/java/org/apache/ignite/cli/commands/cliconfig/CliConfigProfileListCommandTest.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.ignite.cli.commands.cliconfig;
+
+import static org.junit.jupiter.api.Assertions.assertAll;
+
+import org.apache.ignite.cli.commands.cliconfig.profile.CliConfigProfileListCommand;
+import org.apache.ignite.cli.config.ini.IniConfigManager;
+import org.junit.jupiter.api.Test;
+
+class CliConfigProfileListCommandTest extends CliConfigCommandTestBase {
+    @Override
+    protected Class<?> getCommandClass() {
+        return CliConfigProfileListCommand.class;
+    }
+
+    @Test
+    public void testWithDefaultProfile() {
+        execute();
+
+        String expectedResult = "owner" + System.lineSeparator()
+                + "database" + System.lineSeparator();
+        assertAll(
+                () -> assertOutputIs(expectedResult),
+                this::assertErrOutputIsEmpty
+        );
+    }
+
+    @Test
+    public void testSingleProfile() {
+        configManagerProvider.configManager = new IniConfigManager(TestConfigManagerHelper.createClusterUrlNonDefault());
+        execute();
+
+        String expectedResult = "default" + System.lineSeparator();

Review Comment:
   It's a cluster URL which is non-default, not the profile. I'll create a separate single profile ini file.



-- 
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@ignite.apache.org

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


[GitHub] [ignite-3] PakhomovAlexander commented on a diff in pull request #1001: IGNITE-17469 cli profile commands unification

Posted by GitBox <gi...@apache.org>.
PakhomovAlexander commented on code in PR #1001:
URL: https://github.com/apache/ignite-3/pull/1001#discussion_r944801903


##########
modules/cli/src/main/java/org/apache/ignite/cli/commands/cliconfig/CliConfigGetReplCommand.java:
##########
@@ -24,27 +24,23 @@
 import org.apache.ignite.cli.commands.BaseCommand;
 import org.apache.ignite.cli.core.call.CallExecutionPipeline;
 import picocli.CommandLine.Command;
-import picocli.CommandLine.Option;
 import picocli.CommandLine.Parameters;
 
 /**
- * Command to get CLI configuration parameters.
+ * Command to get CLI configuration parameters in REPL mode.
  */
-@Command(name = "get")
-public class CliConfigGetSubCommand extends BaseCommand implements Callable<Integer> {
-    @Parameters
+@Command(name = "get", description = "Gets configuration parameters")
+public class CliConfigGetReplCommand extends BaseCommand implements Callable<Integer> {
+    @Parameters(description = "Property name")
     private String key;
 
-    @Option(names = {"--profile", "-p"}, description = "Get property from specified profile.")
-    private String profileName;
-
     @Inject
     private CliConfigGetCall call;
 
     @Override
     public Integer call() {
         return CallExecutionPipeline.builder(call)
-                .inputProvider(() -> new CliConfigGetCallInput(key, profileName))
+                .inputProvider(() -> new CliConfigGetCallInput(key, null))

Review Comment:
   I would suggest to create one more constructor instead of passing the null.



##########
modules/cli/src/main/java/org/apache/ignite/cli/call/cliconfig/profile/CliConfigProfileListCall.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.ignite.cli.call.cliconfig.profile;
+
+import jakarta.inject.Inject;
+import jakarta.inject.Singleton;
+import java.util.Collection;
+import java.util.stream.Collectors;
+import org.apache.ignite.cli.config.ConfigManagerProvider;
+import org.apache.ignite.cli.core.call.Call;
+import org.apache.ignite.cli.core.call.CallOutput;
+import org.apache.ignite.cli.core.call.DefaultCallOutput;
+import org.apache.ignite.cli.core.call.EmptyCallInput;
+
+/**
+ * List profiles call.
+ */
+@Singleton
+public class CliConfigProfileListCall implements Call<EmptyCallInput, String> {
+
+    @Inject
+    private ConfigManagerProvider provider;
+
+    @Override
+    public CallOutput<String> execute(EmptyCallInput input) {
+        Collection<String> profileNames = provider.get().getProfileNames();
+        String result = profileNames.stream().collect(Collectors.joining(System.lineSeparator()));

Review Comment:
   Shall we mark the active profile with some bold style or, '*' ? 
   Then it will look like:
   * dev
     test
     prod



##########
modules/cli/src/test/java/org/apache/ignite/cli/commands/ProfileMixinTest.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.ignite.cli.commands;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.params.provider.Arguments.arguments;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import io.micronaut.configuration.picocli.MicronautFactory;
+import io.micronaut.context.ApplicationContext;
+import io.micronaut.test.extensions.junit5.annotation.MicronautTest;
+import jakarta.inject.Inject;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import org.apache.ignite.cli.call.cluster.ClusterInitCall;
+import org.apache.ignite.cli.call.cluster.ClusterInitCallInput;
+import org.apache.ignite.cli.call.cluster.topology.LogicalTopologyCall;
+import org.apache.ignite.cli.call.cluster.topology.PhysicalTopologyCall;
+import org.apache.ignite.cli.call.cluster.topology.TopologyCallInput;
+import org.apache.ignite.cli.call.configuration.ClusterConfigShowCall;
+import org.apache.ignite.cli.call.configuration.ClusterConfigShowCallInput;
+import org.apache.ignite.cli.call.configuration.ClusterConfigUpdateCall;
+import org.apache.ignite.cli.call.configuration.ClusterConfigUpdateCallInput;
+import org.apache.ignite.cli.call.configuration.NodeConfigShowCall;
+import org.apache.ignite.cli.call.configuration.NodeConfigShowCallInput;
+import org.apache.ignite.cli.call.configuration.NodeConfigUpdateCall;
+import org.apache.ignite.cli.call.configuration.NodeConfigUpdateCallInput;
+import org.apache.ignite.cli.call.node.status.NodeStatusCall;
+import org.apache.ignite.cli.core.call.Call;
+import org.apache.ignite.cli.core.call.CallInput;
+import org.apache.ignite.cli.core.call.DefaultCallOutput;
+import org.apache.ignite.cli.core.call.StatusCallInput;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.mockito.ArgumentCaptor;
+import picocli.CommandLine;
+
+/**
+ * Test for --profile override for --node-url and --cluster-endpoint-url options.
+ */
+@MicronautTest
+public class ProfileMixinTest {
+    /**
+     * Cluster URL from default profile in integration_tests.ini.
+     */
+    private static final String DEFAULT_URL = "http://localhost:10300";
+
+    /**
+     * Cluster URL from test profile in integration_tests.ini.
+     */
+    private static final String PROFILE_URL = "http://localhost:10301";
+
+    /**
+     * Cluster URL override from command line.
+     */
+    private static final String TEST_URL = "http://localhost:10302";

Review Comment:
   URL_FROM_CMD



##########
modules/cli/src/test/java/org/apache/ignite/cli/commands/cliconfig/CliConfigProfileListCommandTest.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.ignite.cli.commands.cliconfig;
+
+import static org.junit.jupiter.api.Assertions.assertAll;
+
+import org.apache.ignite.cli.commands.cliconfig.profile.CliConfigProfileListCommand;
+import org.junit.jupiter.api.Test;
+
+class CliConfigProfileListCommandTest extends CliConfigCommandTestBase {
+    @Override
+    protected Class<?> getCommandClass() {
+        return CliConfigProfileListCommand.class;
+    }
+
+    @Test
+    public void testWithInternalSection() {

Review Comment:
   Could you add two more test cases?  
   - for the single profile configured
   - if there is no profile configured



##########
modules/cli/src/main/java/org/apache/ignite/cli/commands/node/NodeUrlOptions.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.ignite.cli.commands.node;
+
+import static org.apache.ignite.cli.commands.OptionsConstants.NODE_URL_DESC;
+import static org.apache.ignite.cli.commands.OptionsConstants.NODE_URL_OPTION;
+import static org.apache.ignite.cli.commands.OptionsConstants.PROFILE_OPTION;
+import static org.apache.ignite.cli.commands.OptionsConstants.PROFILE_OPTION_DESC;
+import static org.apache.ignite.cli.commands.OptionsConstants.PROFILE_OPTION_SHORT;
+
+import jakarta.inject.Inject;
+import org.apache.ignite.cli.config.ConfigConstants;
+import org.apache.ignite.cli.config.ConfigManager;
+import org.apache.ignite.cli.config.ConfigManagerProvider;
+import picocli.CommandLine.Option;
+
+/**
+ * Helper class to combine node URL and profile options.
+ */
+public class NodeUrlOptions {
+    /**
+     * Node URL option.
+     */
+    @Option(names = {NODE_URL_OPTION}, description = NODE_URL_DESC)
+    private String nodeUrl;
+
+    /**
+     * Profile to get default values from.
+     */
+    @Option(names = {PROFILE_OPTION, PROFILE_OPTION_SHORT}, description = PROFILE_OPTION_DESC)
+    private String profileName;
+
+    @Inject
+    private ConfigManagerProvider configManagerProvider;
+
+    /**
+     * Gets node URL from either the command line or from the config with specified or default profile.
+     *
+     * @return node URL
+     */
+    public String get() {

Review Comment:
   getNodeUrl



##########
modules/cli/src/main/java/org/apache/ignite/cli/call/cliconfig/profile/CliConfigProfileListCall.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.ignite.cli.call.cliconfig.profile;
+
+import jakarta.inject.Inject;
+import jakarta.inject.Singleton;
+import java.util.Collection;
+import java.util.stream.Collectors;
+import org.apache.ignite.cli.config.ConfigManagerProvider;
+import org.apache.ignite.cli.core.call.Call;
+import org.apache.ignite.cli.core.call.CallOutput;
+import org.apache.ignite.cli.core.call.DefaultCallOutput;
+import org.apache.ignite.cli.core.call.EmptyCallInput;
+
+/**
+ * List profiles call.
+ */
+@Singleton
+public class CliConfigProfileListCall implements Call<EmptyCallInput, String> {
+
+    @Inject
+    private ConfigManagerProvider provider;
+
+    @Override
+    public CallOutput<String> execute(EmptyCallInput input) {
+        Collection<String> profileNames = provider.get().getProfileNames();
+        String result = profileNames.stream().collect(Collectors.joining(System.lineSeparator()));

Review Comment:
   Maybe, it is better to return the list of profiles and create a special decorator for this.



##########
modules/cli/src/test/java/org/apache/ignite/cli/commands/ProfileMixinTest.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.ignite.cli.commands;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.params.provider.Arguments.arguments;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import io.micronaut.configuration.picocli.MicronautFactory;
+import io.micronaut.context.ApplicationContext;
+import io.micronaut.test.extensions.junit5.annotation.MicronautTest;
+import jakarta.inject.Inject;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import org.apache.ignite.cli.call.cluster.ClusterInitCall;
+import org.apache.ignite.cli.call.cluster.ClusterInitCallInput;
+import org.apache.ignite.cli.call.cluster.topology.LogicalTopologyCall;
+import org.apache.ignite.cli.call.cluster.topology.PhysicalTopologyCall;
+import org.apache.ignite.cli.call.cluster.topology.TopologyCallInput;
+import org.apache.ignite.cli.call.configuration.ClusterConfigShowCall;
+import org.apache.ignite.cli.call.configuration.ClusterConfigShowCallInput;
+import org.apache.ignite.cli.call.configuration.ClusterConfigUpdateCall;
+import org.apache.ignite.cli.call.configuration.ClusterConfigUpdateCallInput;
+import org.apache.ignite.cli.call.configuration.NodeConfigShowCall;
+import org.apache.ignite.cli.call.configuration.NodeConfigShowCallInput;
+import org.apache.ignite.cli.call.configuration.NodeConfigUpdateCall;
+import org.apache.ignite.cli.call.configuration.NodeConfigUpdateCallInput;
+import org.apache.ignite.cli.call.node.status.NodeStatusCall;
+import org.apache.ignite.cli.core.call.Call;
+import org.apache.ignite.cli.core.call.CallInput;
+import org.apache.ignite.cli.core.call.DefaultCallOutput;
+import org.apache.ignite.cli.core.call.StatusCallInput;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.mockito.ArgumentCaptor;
+import picocli.CommandLine;
+
+/**
+ * Test for --profile override for --node-url and --cluster-endpoint-url options.
+ */
+@MicronautTest
+public class ProfileMixinTest {
+    /**
+     * Cluster URL from default profile in integration_tests.ini.
+     */
+    private static final String DEFAULT_URL = "http://localhost:10300";
+
+    /**
+     * Cluster URL from test profile in integration_tests.ini.
+     */
+    private static final String PROFILE_URL = "http://localhost:10301";

Review Comment:
   URL_FROM_PROFILE



##########
modules/cli/src/main/java/org/apache/ignite/cli/commands/cluster/ClusterUrlOptions.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.ignite.cli.commands.cluster;
+
+import static org.apache.ignite.cli.commands.OptionsConstants.CLUSTER_URL_DESC;
+import static org.apache.ignite.cli.commands.OptionsConstants.CLUSTER_URL_OPTION;
+import static org.apache.ignite.cli.commands.OptionsConstants.PROFILE_OPTION;
+import static org.apache.ignite.cli.commands.OptionsConstants.PROFILE_OPTION_DESC;
+import static org.apache.ignite.cli.commands.OptionsConstants.PROFILE_OPTION_SHORT;
+
+import jakarta.inject.Inject;
+import org.apache.ignite.cli.config.ConfigConstants;
+import org.apache.ignite.cli.config.ConfigManager;
+import org.apache.ignite.cli.config.ConfigManagerProvider;
+import picocli.CommandLine.Option;
+
+/**
+ * Helper class to combine cluster URL and profile options.
+ */
+public class ClusterUrlOptions {
+    /**
+     * Cluster endpoint URL option.
+     */
+    @Option(names = {CLUSTER_URL_OPTION}, description = CLUSTER_URL_DESC)
+    private String clusterUrl;
+
+    /**
+     * Profile to get default values from.
+     */
+    @Option(names = {PROFILE_OPTION, PROFILE_OPTION_SHORT}, description = PROFILE_OPTION_DESC)
+    private String profileName;
+
+    @Inject
+    private ConfigManagerProvider configManagerProvider;
+
+    /**
+     * Gets cluster URL from either the command line or from the config with specified or default profile.
+     *
+     * @return cluster URL
+     */
+    public String get() {

Review Comment:
   getClusterUrl



##########
modules/cli/src/main/java/org/apache/ignite/cli/commands/cliconfig/profile/CliConfigProfileShowCommand.java:
##########
@@ -28,11 +28,11 @@
 /**
  * Show current profile command.
  */
-@CommandLine.Command(name = "show", description = "Show current default profile.")
-public class CliConfigShowProfileCommand extends BaseCommand implements Callable<Integer> {
+@CommandLine.Command(name = "show", description = "Shows current default profile")

Review Comment:
   "current default" sounds strange, maybe just "current"?



##########
modules/cli/src/main/java/org/apache/ignite/cli/commands/cliconfig/CliConfigShowReplCommand.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cli.commands.cliconfig;
+
+import jakarta.inject.Inject;
+import java.util.concurrent.Callable;
+import org.apache.ignite.cli.call.cliconfig.CliConfigShowCall;
+import org.apache.ignite.cli.commands.BaseCommand;
+import org.apache.ignite.cli.commands.decorators.ProfileDecorator;
+import org.apache.ignite.cli.core.call.CallExecutionPipeline;
+import org.apache.ignite.cli.core.call.StringCallInput;
+import picocli.CommandLine.Command;
+
+/**
+ * Command to get CLI configuration in REPL mode.
+ */
+@Command(name = "show", description = "Shows currently activated config")
+public class CliConfigShowReplCommand extends BaseCommand implements Callable<Integer> {
+    @Inject
+    private CliConfigShowCall call;
+
+    @Override
+    public Integer call() {
+        return CallExecutionPipeline.builder(call)
+                .inputProvider(() -> new StringCallInput(null))

Review Comment:
   Tha same for the constructor.



-- 
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@ignite.apache.org

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