sashapolo commented on code in PR #1569:
URL: https://github.com/apache/ignite-3/pull/1569#discussion_r1109758643


##########
modules/api/src/main/java/org/apache/ignite/Ignition.java:
##########
@@ -171,5 +172,11 @@ CompletableFuture<Ignite> start(
      * @throws IgniteException If the given node has not been started or has 
been stopped.
      * @see <a 
href="https://cwiki.apache.org/confluence/display/IGNITE/IEP-77%3A+Node+Join+Protocol+and+Initialization+for+Ignite+3";>IEP-77</a>
      */
-    void init(String nodeName, Collection<String> metaStorageNodeNames, 
Collection<String> cmgNodeNames, String clusterName);
+    void init(
+            String nodeName,
+            Collection<String> metaStorageNodeNames,
+            Collection<String> cmgNodeNames,
+            String clusterName,
+            RestAuthConfig restAuthConfig

Review Comment:
   Please add new parameter's description to the javadoc



##########
modules/api/src/main/java/org/apache/ignite/Ignition.java:
##########
@@ -149,7 +150,7 @@ CompletableFuture<Ignite> start(
      * @throws IgniteException If the given node has not been started or has 
been stopped.
      * @see <a 
href="https://cwiki.apache.org/confluence/display/IGNITE/IEP-77%3A+Node+Join+Protocol+and+Initialization+for+Ignite+3";>IEP-77</a>
      */
-    void init(String nodeName, Collection<String> metaStorageNodeNames, String 
clusterName);
+    void init(String nodeName, Collection<String> metaStorageNodeNames, String 
clusterName, RestAuthConfig restAuthConfig);

Review Comment:
   Please add new parameter's description to the javadoc



##########
modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterManagementGroupManager.java:
##########
@@ -293,6 +306,7 @@ private void handleInit(CmgInitMessage msg, String 
senderConsistentId, long corr
                                 if (e == null) {
                                     LOG.info("CMG initialized successfully");
 
+

Review Comment:
   Looks like an accidental change



##########
modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/network/auth/AuthProvider.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.internal.cluster.management.network.auth;
+
+import java.io.Serializable;
+import 
org.apache.ignite.internal.cluster.management.network.messages.CmgMessagesFactory;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.rest.AuthProviderConfig;
+import org.apache.ignite.rest.BasicAuthProviderConfig;
+
+/** Represents authentication provider config. */
+public interface AuthProvider extends NetworkMessage, Serializable {
+
+    /** Type of the provider. */
+    String type();
+
+    String name();

Review Comment:
   missing javadoc



##########
modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterState.java:
##########
@@ -64,6 +65,8 @@ default IgniteProductVersion igniteVersion() {
         return IgniteProductVersion.fromString(version());
     }
 
+    RestAuth restAuthToApply();

Review Comment:
   Missing javadoc



##########
modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/network/messages/CmgMessageGroup.java:
##########
@@ -117,5 +117,11 @@ public interface Commands  {
          * Message type for {@link ClusterTag}.
          */
         int CLUSTER_TAG = 62;
+
+        int REST_AUTH = 63;

Review Comment:
   Please update the javadocs like all other fields



##########
modules/rest/src/main/java/org/apache/ignite/internal/rest/RestComponent.java:
##########
@@ -190,8 +193,11 @@ private BindException 
findBindException(ApplicationStartupException e) {
     private Micronaut buildMicronautContext(int portCandidate, int 
sslPortCandidate) {
         Micronaut micronaut = Micronaut.build("");
         setFactories(micronaut);
+        Map<String, Object> properties = 
Stream.of(serverProperties(portCandidate, sslPortCandidate), authProperties())

Review Comment:
   I think that simply creating a `HashMap` and using `putAll` is a more 
readable approach than using streams here



##########
modules/rest-api/src/main/java/org/apache/ignite/internal/rest/api/cluster/InitCommand.java:
##########
@@ -41,14 +42,17 @@ public class InitCommand {
     @Schema(description = "The name of the cluster.")
     private final String clusterName;
 
+    private final AuthConfigDto authConfig;

Review Comment:
   Shouldn't this field also have the `@Schema` annotation?



##########
modules/rest/src/main/java/org/apache/ignite/internal/rest/auth/AuthProviderFactory.java:
##########
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.rest.auth;
+
+import io.micronaut.context.annotation.Bean;
+import io.micronaut.context.annotation.Factory;
+import io.micronaut.security.authentication.AuthenticationProvider;
+import jakarta.inject.Singleton;
+import org.apache.ignite.internal.rest.RestFactory;
+import org.apache.ignite.internal.rest.configuration.AuthConfiguration;
+
+/**
+ * Factory that creates beans that are needed for authentication.
+ */
+@Factory
+public class AuthProviderFactory implements RestFactory {
+
+    private final DelegatingAuthenticationProvider authenticationProvider;
+
+    public AuthProviderFactory(AuthConfiguration authConfiguration) {
+        this.authenticationProvider = new DelegatingAuthenticationProvider();

Review Comment:
   Is this a correct way to use Bean Factories? What's the point of having a 
factory that creates singletons if this singleton is cached inside the factory 
itself? (I'm just asking, I'm not a Micronaut expert)



##########
modules/rest/src/main/java/org/apache/ignite/internal/rest/auth/Authenticator.java:
##########
@@ -0,0 +1,25 @@
+/*
+ * 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.internal.rest.auth;
+
+import io.micronaut.security.authentication.AuthenticationRequest;
+import io.micronaut.security.authentication.AuthenticationResponse;
+
+interface Authenticator {

Review Comment:
   Missing documentation



##########
modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterInitializer.java:
##########
@@ -66,7 +68,8 @@ public ClusterInitializer(ClusterService clusterService) {
     public CompletableFuture<Void> initCluster(
             Collection<String> metaStorageNodeNames,
             Collection<String> cmgNodeNames,
-            String clusterName
+            String clusterName,
+            RestAuthConfig restAuthConfig

Review Comment:
   Missing javadoc description



##########
modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterManagementGroupManager.java:
##########
@@ -366,6 +381,32 @@ private void onElectedAsLeader(long term) {
                         LOG.warn("Error when executing onLeaderElected 
callback", e);
                     }
                 });
+
+        raftServiceAfterJoin().thenComposeAsync(service -> {

Review Comment:
   Can you please explain what are you trying to do here? I'm sorry, but this 
code makes no sense to me...



##########
modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterManagementGroupManager.java:
##########
@@ -796,6 +845,32 @@ private CompletableFuture<CmgRaftService> 
raftServiceAfterJoin() {
                 });
     }
 
+    private static CompletableFuture<Void> 
changeAuthConfiguration(AuthConfiguration authConfiguration, RestAuth restAuth) 
{
+        return authConfiguration.change(authChange -> {
+            authChange.changeProviders(providers -> {
+                restAuth.providers().forEach(provider -> {
+                    providers.create(provider.name(), cfg -> 
applyProviderChange(cfg, provider));
+                });
+            });
+            authChange.changeEnabled(restAuth.enabled());
+        }).whenComplete((v, e) -> {
+            LOG.error("Unable to change auth configuration", e);

Review Comment:
   You forgot the `null` check for `e`



##########
modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterManagementGroupManager.java:
##########
@@ -580,6 +621,14 @@ public void onDisappeared(ClusterNode member) {
         };
     }
 
+    private CompletableFuture<Void> 
applyRestConfigurationChanges(CmgRaftService raftService) {
+        return raftService.readClusterState()
+                .thenCombine(clusterRestConfigurationFuture, (clusterState, 
restConfiguration) -> {

Review Comment:
   You should use `thenAcceptBoth` here



##########
modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/raft/CmgRaftGroupListener.java:
##########
@@ -124,6 +125,10 @@ public void onWrite(Iterator<CommandClosure<WriteCommand>> 
iterator) {
                 Serializable response = initCmgState((InitCmgStateCommand) 
command);
 
                 clo.result(response);
+            } else if (command instanceof UpdateClusterStateCommand) {
+                UpdateClusterStateCommand updateClusterStateCommand = 
(UpdateClusterStateCommand) command;
+                
storage.putClusterState(updateClusterStateCommand.clusterState());
+                clo.result(updateClusterStateCommand.clusterState());

Review Comment:
   Why do you need to return the cluster state as a response?



##########
modules/rest/src/main/java/org/apache/ignite/internal/rest/configuration/AuthProviderConfigurationSchema.java:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.internal.rest.configuration;
+
+import org.apache.ignite.configuration.annotation.PolymorphicConfig;
+import org.apache.ignite.configuration.annotation.PolymorphicId;
+import org.apache.ignite.configuration.annotation.Value;
+
+/** Auth provider configuration. */
+@PolymorphicConfig
+public class AuthProviderConfigurationSchema {
+
+    public static final String TYPE_BASIC = "basic";
+
+    /** Auth type. */
+    @PolymorphicId
+    public String type;
+
+
+    @Value
+    public String name;

Review Comment:
   What is this field for?



##########
modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterManagementGroupManager.java:
##########
@@ -796,6 +845,32 @@ private CompletableFuture<CmgRaftService> 
raftServiceAfterJoin() {
                 });
     }
 
+    private static CompletableFuture<Void> 
changeAuthConfiguration(AuthConfiguration authConfiguration, RestAuth restAuth) 
{

Review Comment:
   I think that this logic of applying `RestAuth` to `AuthConfiguration` should 
be extracted elsewhere. For example, into a separate class or into `RestAuth` 
itself



##########
modules/rest/src/test/java/org/apache/ignite/internal/rest/auth/TestAuthSubscriber.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.internal.rest.auth;
+
+import io.micronaut.security.authentication.AuthenticationResponse;
+import org.jetbrains.annotations.Nullable;
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.Subscription;
+
+/** Implementation of {@link Subscriber} for tests. */
+public class TestAuthSubscriber implements Subscriber<AuthenticationResponse> {
+    private Subscription source;
+    private @Nullable AuthenticationResponse lastResponse;
+    private @Nullable Throwable lastError;
+
+    @Override
+    public void onSubscribe(Subscription subscription) {
+        source = subscription;
+        source.request(1);
+    }
+
+    @Override
+    public void onNext(AuthenticationResponse authenticationResponse) {
+        lastResponse = authenticationResponse;
+    }
+
+    @Override
+    public void onError(Throwable throwable) {
+        lastError = throwable;
+    }
+
+    @Override
+    public void onComplete() {
+        source.cancel();
+    }
+
+    public void reset() {

Review Comment:
   why do you need this method? Can you simply create new subscribers if needed?



##########
modules/rest/src/test/java/org/apache/ignite/internal/rest/auth/TestAuthSubscriber.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.internal.rest.auth;
+
+import io.micronaut.security.authentication.AuthenticationResponse;
+import org.jetbrains.annotations.Nullable;
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.Subscription;
+
+/** Implementation of {@link Subscriber} for tests. */
+public class TestAuthSubscriber implements Subscriber<AuthenticationResponse> {
+    private Subscription source;
+    private @Nullable AuthenticationResponse lastResponse;
+    private @Nullable Throwable lastError;
+
+    @Override
+    public void onSubscribe(Subscription subscription) {
+        source = subscription;
+        source.request(1);
+    }
+
+    @Override
+    public void onNext(AuthenticationResponse authenticationResponse) {
+        lastResponse = authenticationResponse;
+    }
+
+    @Override
+    public void onError(Throwable throwable) {
+        lastError = throwable;
+    }
+
+    @Override
+    public void onComplete() {
+        source.cancel();
+    }
+
+    public void reset() {
+        lastResponse = null;
+        lastError = null;
+    }
+
+    public AuthenticationResponse lastResponse() {
+        return lastResponse;

Review Comment:
   Can this field be read from a different thread that was used to call 
`onNext`? Looking at the usage I can guess that yes. Should we at least mark 
`lastResponse` and `lastError` as volatile?



##########
modules/rest/src/test/java/org/apache/ignite/internal/rest/auth/TestAuthSubscriber.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.internal.rest.auth;
+
+import io.micronaut.security.authentication.AuthenticationResponse;
+import org.jetbrains.annotations.Nullable;
+import org.reactivestreams.Subscriber;
+import org.reactivestreams.Subscription;
+
+/** Implementation of {@link Subscriber} for tests. */
+public class TestAuthSubscriber implements Subscriber<AuthenticationResponse> {
+    private Subscription source;
+    private @Nullable AuthenticationResponse lastResponse;
+    private @Nullable Throwable lastError;
+
+    @Override
+    public void onSubscribe(Subscription subscription) {
+        source = subscription;
+        source.request(1);
+    }
+
+    @Override
+    public void onNext(AuthenticationResponse authenticationResponse) {
+        lastResponse = authenticationResponse;
+    }
+
+    @Override
+    public void onError(Throwable throwable) {
+        lastError = throwable;
+    }
+
+    @Override
+    public void onComplete() {
+        source.cancel();

Review Comment:
   Why do you cancel the subscription here? It's already complete



##########
modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/ClusterManagementGroupManager.java:
##########
@@ -366,6 +381,32 @@ private void onElectedAsLeader(long term) {
                         LOG.warn("Error when executing onLeaderElected 
callback", e);
                     }
                 });
+
+        raftServiceAfterJoin().thenComposeAsync(service -> {

Review Comment:
   Also, if you are trying to apply the Rest configuration in `onLeaderElected` 
callback, how is it going to work when a new leader is elected? Are we going to 
apply the configuration again and fail?



##########
modules/rest/src/main/java/org/apache/ignite/internal/rest/auth/DelegatingAuthenticationProvider.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.internal.rest.auth;
+
+import io.micronaut.http.HttpRequest;
+import io.micronaut.security.authentication.AuthenticationProvider;
+import io.micronaut.security.authentication.AuthenticationRequest;
+import io.micronaut.security.authentication.AuthenticationResponse;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.notifications.ConfigurationListener;
+import 
org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.rest.configuration.AuthConfiguration;
+import org.apache.ignite.internal.rest.configuration.AuthProviderView;
+import org.apache.ignite.internal.rest.configuration.AuthView;
+import org.jetbrains.annotations.Nullable;
+import org.reactivestreams.Publisher;
+import reactor.core.publisher.Flux;
+import reactor.core.publisher.FluxSink;
+
+/**
+ * Implementation of {@link AuthenticationProvider}. Creates a list of {@link 
Authenticator} according to provided {@link AuthConfiguration}
+ * and updates them on configuration changes. Delegates {@link 
AuthenticationRequest} to the list of {@link Authenticator}.
+ */
+public class DelegatingAuthenticationProvider implements 
AuthenticationProvider, ConfigurationListener<AuthView> {
+
+    private static final IgniteLogger LOG = 
Loggers.forClass(DelegatingAuthenticationProvider.class);
+
+    private final AtomicBoolean authEnabled = new AtomicBoolean(false);
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Lock readLock = rwLock.readLock();

Review Comment:
   I don't think having these fields is necessary. It is a common practice to 
write `rwLock.readLock().lock()`



##########
modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/network/messages/CmgInitMessage.java:
##########
@@ -40,4 +41,6 @@ public interface CmgInitMessage extends NetworkMessage {
      * Name of the cluster that will be a part of the generated cluster tag.
      */
     String clusterName();
+
+    RestAuth restAuth();

Review Comment:
   missing javadoc



##########
modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java:
##########
@@ -253,6 +257,9 @@ public class IgniteImpl implements Ignite {
 
     private final RestAddressReporter restAddressReporter;
 
+    /** CompletableFuture of {@link ClusterRestConfiguration} that will be 
resolved after the cluster initialization. */
+    private final CompletableFuture<ClusterRestConfiguration> 
clusterRestConfigurationFuture = new CompletableFuture<>();

Review Comment:
   I think that this logic should be encapsulated in a separate class, for 
example, `DistributedConfigurationUpdater`. This class will know how to apply 
the AuthConfiguration (which will be provided by the CMG Manager) after all 
components have started. What do you think?



##########
modules/rest/src/main/java/org/apache/ignite/internal/rest/auth/DelegatingAuthenticationProvider.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.internal.rest.auth;
+
+import io.micronaut.http.HttpRequest;
+import io.micronaut.security.authentication.AuthenticationProvider;
+import io.micronaut.security.authentication.AuthenticationRequest;
+import io.micronaut.security.authentication.AuthenticationResponse;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.notifications.ConfigurationListener;
+import 
org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.rest.configuration.AuthConfiguration;
+import org.apache.ignite.internal.rest.configuration.AuthProviderView;
+import org.apache.ignite.internal.rest.configuration.AuthView;
+import org.jetbrains.annotations.Nullable;
+import org.reactivestreams.Publisher;
+import reactor.core.publisher.Flux;
+import reactor.core.publisher.FluxSink;
+
+/**
+ * Implementation of {@link AuthenticationProvider}. Creates a list of {@link 
Authenticator} according to provided {@link AuthConfiguration}
+ * and updates them on configuration changes. Delegates {@link 
AuthenticationRequest} to the list of {@link Authenticator}.
+ */
+public class DelegatingAuthenticationProvider implements 
AuthenticationProvider, ConfigurationListener<AuthView> {
+
+    private static final IgniteLogger LOG = 
Loggers.forClass(DelegatingAuthenticationProvider.class);
+
+    private final AtomicBoolean authEnabled = new AtomicBoolean(false);
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Lock readLock = rwLock.readLock();
+    private final Lock writeLock = rwLock.writeLock();
+
+    private List<Authenticator> authenticators = new CopyOnWriteArrayList<>();

Review Comment:
   ```suggestion
       private final List<Authenticator> authenticators = new 
CopyOnWriteArrayList<>();
   ```



##########
modules/cluster-management/src/test/java/org/apache/ignite/internal/cluster/management/ClusterInitializerTest.java:
##########
@@ -210,21 +215,25 @@ private CompletableFuture<NetworkMessage> 
initCompleteMessage() {
      */
     @Test
     void testInitIllegalArguments() {
-        assertThrows(IllegalArgumentException.class, () -> 
clusterInitializer.initCluster(List.of(), List.of(), "cluster"));
+        assertThrows(IllegalArgumentException.class,
+                () -> clusterInitializer.initCluster(List.of(), List.of(), 
"cluster", disabledAuth()));
 
-        assertThrows(IllegalArgumentException.class, () -> 
clusterInitializer.initCluster(List.of(" "), List.of("bar"), "cluster"));
+        assertThrows(IllegalArgumentException.class,
+                () -> clusterInitializer.initCluster(List.of(" "), 
List.of("bar"), "cluster", disabledAuth()));
 
-        assertThrows(IllegalArgumentException.class, () -> 
clusterInitializer.initCluster(List.of("foo"), List.of(" "), "cluster"));
+        assertThrows(IllegalArgumentException.class,
+                () -> clusterInitializer.initCluster(List.of("foo"), List.of(" 
"), "cluster", disabledAuth()));

Review Comment:
   Looks like we need an override for the `initCluster` method with auth 
disabled by default



##########
modules/rest/src/main/java/org/apache/ignite/internal/rest/auth/DelegatingAuthenticationProvider.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.internal.rest.auth;
+
+import io.micronaut.http.HttpRequest;
+import io.micronaut.security.authentication.AuthenticationProvider;
+import io.micronaut.security.authentication.AuthenticationRequest;
+import io.micronaut.security.authentication.AuthenticationResponse;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.notifications.ConfigurationListener;
+import 
org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.rest.configuration.AuthConfiguration;
+import org.apache.ignite.internal.rest.configuration.AuthProviderView;
+import org.apache.ignite.internal.rest.configuration.AuthView;
+import org.jetbrains.annotations.Nullable;
+import org.reactivestreams.Publisher;
+import reactor.core.publisher.Flux;
+import reactor.core.publisher.FluxSink;
+
+/**
+ * Implementation of {@link AuthenticationProvider}. Creates a list of {@link 
Authenticator} according to provided {@link AuthConfiguration}
+ * and updates them on configuration changes. Delegates {@link 
AuthenticationRequest} to the list of {@link Authenticator}.
+ */
+public class DelegatingAuthenticationProvider implements 
AuthenticationProvider, ConfigurationListener<AuthView> {
+
+    private static final IgniteLogger LOG = 
Loggers.forClass(DelegatingAuthenticationProvider.class);
+
+    private final AtomicBoolean authEnabled = new AtomicBoolean(false);
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Lock readLock = rwLock.readLock();
+    private final Lock writeLock = rwLock.writeLock();
+
+    private List<Authenticator> authenticators = new CopyOnWriteArrayList<>();
+
+    public DelegatingAuthenticationProvider() {}
+
+    @Override
+    public Publisher<AuthenticationResponse> authenticate(HttpRequest<?> 
httpRequest, AuthenticationRequest<?, ?> authenticationRequest) {
+        return Flux.create(emitter -> {
+            readLock.lock();
+            try {
+                if (authEnabled.get()) {
+                    Optional<AuthenticationResponse> successResponse = 
authenticators.stream()
+                            .map(it -> it.authenticate(authenticationRequest))
+                            .filter(AuthenticationResponse::isAuthenticated)
+                            .findFirst();
+                    if (successResponse.isPresent()) {
+                        emitter.next(successResponse.get());
+                        emitter.complete();
+                    } else {
+                        emitter.error(AuthenticationResponse.exception());
+                    }
+                } else {
+                    emitter.next(AuthenticationResponse.success("Unknown"));
+                    emitter.complete();
+                }
+            } finally {
+                readLock.unlock();
+            }
+        }, FluxSink.OverflowStrategy.ERROR);
+    }
+
+    @Override
+    public CompletableFuture<?> 
onUpdate(ConfigurationNotificationEvent<AuthView> ctx) {
+        return CompletableFuture.runAsync(() -> 
refreshProviders(ctx.newValue()));
+    }
+
+    public boolean authEnabled() {
+        return authEnabled.get();
+    }
+
+    private void refreshProviders(@Nullable AuthView view) {
+        writeLock.lock();
+        try {
+            if (view == null || !view.enabled()) {
+                authEnabled.set(false);

Review Comment:
   Why do we need the `authEnabled` field? We can identify that authentication 
is disabled if the `authenticators` list is empty



##########
modules/rest/src/main/java/org/apache/ignite/internal/rest/auth/DelegatingAuthenticationProvider.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.internal.rest.auth;
+
+import io.micronaut.http.HttpRequest;
+import io.micronaut.security.authentication.AuthenticationProvider;
+import io.micronaut.security.authentication.AuthenticationRequest;
+import io.micronaut.security.authentication.AuthenticationResponse;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.notifications.ConfigurationListener;
+import 
org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.rest.configuration.AuthConfiguration;
+import org.apache.ignite.internal.rest.configuration.AuthProviderView;
+import org.apache.ignite.internal.rest.configuration.AuthView;
+import org.jetbrains.annotations.Nullable;
+import org.reactivestreams.Publisher;
+import reactor.core.publisher.Flux;
+import reactor.core.publisher.FluxSink;
+
+/**
+ * Implementation of {@link AuthenticationProvider}. Creates a list of {@link 
Authenticator} according to provided {@link AuthConfiguration}
+ * and updates them on configuration changes. Delegates {@link 
AuthenticationRequest} to the list of {@link Authenticator}.
+ */
+public class DelegatingAuthenticationProvider implements 
AuthenticationProvider, ConfigurationListener<AuthView> {
+
+    private static final IgniteLogger LOG = 
Loggers.forClass(DelegatingAuthenticationProvider.class);
+
+    private final AtomicBoolean authEnabled = new AtomicBoolean(false);
+    private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+    private final Lock readLock = rwLock.readLock();
+    private final Lock writeLock = rwLock.writeLock();
+
+    private List<Authenticator> authenticators = new CopyOnWriteArrayList<>();

Review Comment:
   Why is this a `CopyOnWriteArrayList` if you always access it either under a 
`readLock` or a `writeLock`?



##########
modules/rest/src/main/java/org/apache/ignite/internal/rest/configuration/AuthConfigurationSchema.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.internal.rest.configuration;
+
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.NamedConfigValue;
+import org.apache.ignite.configuration.annotation.Value;
+
+/**
+ * Configuration schema for auth endpoint subtree.
+ */
+@SuppressWarnings("PMD.UnusedPrivateField")
+@Config
+public class AuthConfigurationSchema {
+
+    /** Enabled. */
+    @Value(hasDefault = true)
+    public final boolean enabled = false;
+
+    /** Auth configurations. */
+    @NamedConfigValue
+    @AuthProvidersValidator
+    public AuthProviderConfigurationSchema providers;

Review Comment:
   Why do we need multiple auth configurations?



##########
modules/rest/src/main/java/org/apache/ignite/internal/rest/configuration/AuthConfigurationValidator.java:
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.internal.rest.configuration;
+
+import static java.lang.annotation.ElementType.FIELD;
+import static java.lang.annotation.RetentionPolicy.RUNTIME;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.Target;
+
+/**
+ * Annotation to validate whole auth configuration.
+ *
+ * <p>Activate AuthConfigurationValidatorImpl in configuration engine for 
{@link ClusterRestConfigurationSchema#authConfiguration}.

Review Comment:
   What does this mean?



##########
modules/rest/src/main/java/org/apache/ignite/internal/rest/configuration/AuthProviderConfigurationSchema.java:
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.internal.rest.configuration;
+
+import org.apache.ignite.configuration.annotation.PolymorphicConfig;
+import org.apache.ignite.configuration.annotation.PolymorphicId;
+import org.apache.ignite.configuration.annotation.Value;
+
+/** Auth provider configuration. */
+@PolymorphicConfig
+public class AuthProviderConfigurationSchema {
+
+    public static final String TYPE_BASIC = "basic";
+
+    /** Auth type. */
+    @PolymorphicId
+    public String type;
+
+

Review Comment:
   too many empty lines



##########
modules/rest/src/main/java/org/apache/ignite/internal/rest/auth/DelegatingAuthenticationProvider.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.internal.rest.auth;
+
+import io.micronaut.http.HttpRequest;
+import io.micronaut.security.authentication.AuthenticationProvider;
+import io.micronaut.security.authentication.AuthenticationRequest;
+import io.micronaut.security.authentication.AuthenticationResponse;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.notifications.ConfigurationListener;
+import 
org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.rest.configuration.AuthConfiguration;
+import org.apache.ignite.internal.rest.configuration.AuthProviderView;
+import org.apache.ignite.internal.rest.configuration.AuthView;
+import org.jetbrains.annotations.Nullable;
+import org.reactivestreams.Publisher;
+import reactor.core.publisher.Flux;
+import reactor.core.publisher.FluxSink;
+
+/**
+ * Implementation of {@link AuthenticationProvider}. Creates a list of {@link 
Authenticator} according to provided {@link AuthConfiguration}
+ * and updates them on configuration changes. Delegates {@link 
AuthenticationRequest} to the list of {@link Authenticator}.
+ */
+public class DelegatingAuthenticationProvider implements 
AuthenticationProvider, ConfigurationListener<AuthView> {
+
+    private static final IgniteLogger LOG = 
Loggers.forClass(DelegatingAuthenticationProvider.class);
+
+    private final AtomicBoolean authEnabled = new AtomicBoolean(false);

Review Comment:
   Same here, this variable doesn't have to be an `AtomicBoolean`, you can 
simply guard the access with a lock



##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/rest/RestNode.java:
##########
@@ -15,30 +15,34 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.rest.ssl;
+package org.apache.ignite.internal.rest;
 
 import java.nio.file.Path;
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.Ignite;
 import org.apache.ignite.IgnitionManager;
+import org.apache.ignite.internal.rest.ssl.ItRestSslTest;
 
 /** Presentation of Ignite node for tests. */
 public class RestNode {
 
-    /** Key store path. */
-    private static final String keyStorePath = "ssl/keystore.p12";
-
-    /** Key store password. */
-    private static final String keyStorePassword = "changeit";
-
+    private final String keyStorePath;
+    private final String keyStorePassword;
     private final Path workDir;
     private final String name;
     private final int networkPort;
     private final int httpPort;
     private final int httpsPort;
     private final boolean sslEnabled;
     private final boolean dualProtocol;
+    private CompletableFuture<Ignite> igniteNodeFuture;
+
+

Review Comment:
   too many empty lines



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to