External Infinispan as cache - Part 3

Implementation of UserLoginFailureProvider using remote caches only.

Closes #28754

Signed-off-by: Pedro Ruivo <pruivo@redhat.com>
This commit is contained in:
Pedro Ruivo 2024-04-24 13:58:44 +02:00 committed by Alexander Schwartz
parent 833aad661e
commit 9006218559
48 changed files with 1787 additions and 217 deletions

View file

@ -358,6 +358,46 @@ jobs:
name: store-it-mvn-logs
path: .github/scripts/ansible/files
external-infinispan-tests:
name: External Infinispan IT
needs: [ build, conditional ]
if: needs.conditional.outputs.ci-store == 'true'
runs-on: ubuntu-latest
timeout-minutes: 150
strategy:
matrix:
variant: [ "remote-cache,multi-site" ]
fail-fast: false
steps:
- uses: actions/checkout@v4
- id: integration-test-setup
name: Integration test setup
uses: ./.github/actions/integration-test-setup
- name: Run base tests without cache
run: |
TESTS=`testsuite/integration-arquillian/tests/base/testsuites/suite.sh persistent-sessions`
echo "Tests: $TESTS"
./mvnw test ${{ env.SUREFIRE_RETRY }} -Pauth-server-quarkus -Pinfinispan-server -Dauth.server.feature=${{ matrix.variant }} -Dtest=$TESTS -pl testsuite/integration-arquillian/tests/base 2>&1 | misc/log/trimmer.sh
- name: Upload JVM Heapdumps
if: always()
uses: ./.github/actions/upload-heapdumps
- uses: ./.github/actions/upload-flaky-tests
name: Upload flaky tests
env:
GH_TOKEN: ${{ github.token }}
with:
job-name: Remote Infinispan IT
- name: Surefire reports
if: always()
uses: ./.github/actions/archive-surefire-reports
with:
job-id: remote-infinispan-integration-tests
store-integration-tests:
name: Store IT
needs: [build, conditional]
@ -823,6 +863,7 @@ jobs:
- webauthn-integration-tests
- sssd-unit-tests
- migration-tests
- external-infinispan-tests
runs-on: ubuntu-latest
steps:
- uses: actions/checkout@v4

View file

@ -17,6 +17,14 @@
package org.keycloak.cluster.infinispan;
import java.util.Collection;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
import org.infinispan.Cache;
import org.infinispan.client.hotrod.exceptions.HotRodClientException;
import org.infinispan.lifecycle.ComponentStatus;
@ -38,21 +46,14 @@ import org.keycloak.connections.infinispan.TopologyInfo;
import org.keycloak.infinispan.util.InfinispanUtils;
import org.keycloak.models.KeycloakSession;
import org.keycloak.models.KeycloakSessionFactory;
import java.util.Collection;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
import org.keycloak.provider.EnvironmentDependentProviderFactory;
/**
* This impl is aware of Cross-Data-Center scenario too
*
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
*/
public class InfinispanClusterProviderFactory implements ClusterProviderFactory {
public class InfinispanClusterProviderFactory implements ClusterProviderFactory, EnvironmentDependentProviderFactory {
protected static final Logger logger = Logger.getLogger(InfinispanClusterProviderFactory.class);

View file

@ -1,15 +1,21 @@
package org.keycloak.cluster.infinispan.remote;
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.
*/
import org.infinispan.client.hotrod.RemoteCache;
import org.jboss.logging.Logger;
import org.keycloak.cluster.ClusterEvent;
import org.keycloak.cluster.ClusterListener;
import org.keycloak.cluster.ClusterProvider;
import org.keycloak.cluster.ExecutionResult;
import org.keycloak.cluster.infinispan.LockEntry;
import org.keycloak.cluster.infinispan.TaskCallback;
import org.keycloak.common.util.Retry;
import org.keycloak.common.util.Time;
package org.keycloak.cluster.infinispan.remote;
import java.lang.invoke.MethodHandles;
import java.util.Collection;
@ -22,27 +28,31 @@ import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
import org.infinispan.client.hotrod.RemoteCache;
import org.jboss.logging.Logger;
import org.keycloak.cluster.ClusterEvent;
import org.keycloak.cluster.ClusterListener;
import org.keycloak.cluster.ClusterProvider;
import org.keycloak.cluster.ExecutionResult;
import org.keycloak.cluster.infinispan.LockEntry;
import org.keycloak.cluster.infinispan.TaskCallback;
import org.keycloak.common.util.Retry;
import static org.keycloak.cluster.infinispan.InfinispanClusterProvider.TASK_KEY_PREFIX;
import static org.keycloak.cluster.infinispan.remote.RemoteInfinispanClusterProviderFactory.putIfAbsentWithRetries;
public class RemoteInfinispanClusterProvider implements ClusterProvider {
private static final Logger logger = Logger.getLogger(MethodHandles.lookup().lookupClass());
private final int clusterStartupTime;
private final RemoteCache<String, LockEntry> cache;
private final RemoteInfinispanNotificationManager notificationManager;
private final Executor executor;
private final SharedData data;
public RemoteInfinispanClusterProvider(int clusterStartupTime, RemoteCache<String, LockEntry> cache, RemoteInfinispanNotificationManager notificationManager, Executor executor) {
this.clusterStartupTime = clusterStartupTime;
this.cache = Objects.requireNonNull(cache);
this.notificationManager = Objects.requireNonNull(notificationManager);
this.executor = Objects.requireNonNull(executor);
public RemoteInfinispanClusterProvider(SharedData data) {
this.data = Objects.requireNonNull(data);
}
@Override
public int getClusterStartupTime() {
return clusterStartupTime;
return data.clusterStartupTime();
}
@Override
@ -70,7 +80,7 @@ public class RemoteInfinispanClusterProvider implements ClusterProvider {
@Override
public Future<Boolean> executeIfNotExecutedAsync(String taskKey, int taskTimeoutInSeconds, Callable task) {
TaskCallback newCallback = new TaskCallback();
TaskCallback callback = notificationManager.registerTaskCallback(TASK_KEY_PREFIX + taskKey, newCallback);
TaskCallback callback = data.notificationManager().registerTaskCallback(TASK_KEY_PREFIX + taskKey, newCallback);
// We successfully submitted our task
if (newCallback == callback) {
@ -89,7 +99,7 @@ public class RemoteInfinispanClusterProvider implements ClusterProvider {
return callback.isSuccess();
};
callback.setFuture(CompletableFuture.supplyAsync(wrappedTask, executor));
callback.setFuture(CompletableFuture.supplyAsync(wrappedTask, data.executor()));
} else {
logger.infof("Task already in progress on this cluster node. Will wait until it's finished");
}
@ -99,17 +109,17 @@ public class RemoteInfinispanClusterProvider implements ClusterProvider {
@Override
public void registerListener(String taskKey, ClusterListener task) {
notificationManager.registerListener(taskKey, task);
data.notificationManager().registerListener(taskKey, task);
}
@Override
public void notify(String taskKey, ClusterEvent event, boolean ignoreSender, DCNotify dcNotify) {
notificationManager.notify(taskKey, Collections.singleton(event), ignoreSender, dcNotify);
data.notificationManager().notify(taskKey, Collections.singleton(event), ignoreSender, dcNotify);
}
@Override
public void notify(String taskKey, Collection<? extends ClusterEvent> events, boolean ignoreSender, DCNotify dcNotify) {
notificationManager.notify(taskKey, events, ignoreSender, dcNotify);
data.notificationManager().notify(taskKey, events, ignoreSender, dcNotify);
}
@Override
@ -120,7 +130,7 @@ public class RemoteInfinispanClusterProvider implements ClusterProvider {
private boolean tryLock(String cacheKey, int taskTimeoutInSeconds) {
LockEntry myLock = createLockEntry();
LockEntry existingLock = putIfAbsentWithRetries(cache, cacheKey, myLock, taskTimeoutInSeconds);
LockEntry existingLock = putIfAbsentWithRetries(data.cache(), cacheKey, myLock, taskTimeoutInSeconds);
if (existingLock != null) {
if (logger.isTraceEnabled()) {
logger.tracef("Task %s in progress already by node %s. Ignoring task.", cacheKey, existingLock.node());
@ -135,16 +145,23 @@ public class RemoteInfinispanClusterProvider implements ClusterProvider {
}
private LockEntry createLockEntry() {
return new LockEntry(notificationManager.getMyNodeName());
return new LockEntry(data.notificationManager().getMyNodeName());
}
private void removeFromCache(String cacheKey) {
// More attempts to send the message (it may fail if some node fails in the meantime)
Retry.executeWithBackoff((int iteration) -> {
cache.remove(cacheKey);
data.cache().remove(cacheKey);
if (logger.isTraceEnabled()) {
logger.tracef("Task %s removed from the cache", cacheKey);
}
}, 10, 10);
}
public interface SharedData {
int clusterStartupTime();
RemoteCache<String, LockEntry> cache();
RemoteInfinispanNotificationManager notificationManager();
Executor executor();
}
}

View file

@ -1,5 +1,26 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.cluster.infinispan.remote;
import java.lang.invoke.MethodHandles;
import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit;
import org.infinispan.client.hotrod.RemoteCache;
import org.infinispan.client.hotrod.exceptions.HotRodClientException;
import org.infinispan.commons.util.ByRef;
@ -12,19 +33,14 @@ import org.keycloak.cluster.infinispan.LockEntry;
import org.keycloak.common.util.Retry;
import org.keycloak.common.util.Time;
import org.keycloak.connections.infinispan.InfinispanConnectionProvider;
import org.keycloak.connections.infinispan.TopologyInfo;
import org.keycloak.infinispan.util.InfinispanUtils;
import org.keycloak.models.KeycloakSession;
import org.keycloak.models.KeycloakSessionFactory;
import java.io.Serializable;
import java.lang.invoke.MethodHandles;
import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit;
import org.keycloak.provider.EnvironmentDependentProviderFactory;
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.WORK_CACHE_NAME;
public class RemoteInfinispanClusterProviderFactory implements ClusterProviderFactory {
public class RemoteInfinispanClusterProviderFactory implements ClusterProviderFactory, RemoteInfinispanClusterProvider.SharedData, EnvironmentDependentProviderFactory {
private static final Logger logger = Logger.getLogger(MethodHandles.lookup().lookupClass());
@ -35,10 +51,14 @@ public class RemoteInfinispanClusterProviderFactory implements ClusterProviderFa
@Override
public ClusterProvider create(KeycloakSession session) {
if (workCache == null) {
// Keycloak does not ensure postInit() is invoked before create()
lazyInit(session);
}
assert workCache != null;
assert notificationManager != null;
assert executor != null;
return new RemoteInfinispanClusterProvider(clusterStartupTime, workCache, notificationManager, executor);
return new RemoteInfinispanClusterProvider(this);
}
@Override
@ -47,16 +67,9 @@ public class RemoteInfinispanClusterProviderFactory implements ClusterProviderFa
}
@Override
public synchronized void postInit(KeycloakSessionFactory factory) {
public void postInit(KeycloakSessionFactory factory) {
try (var session = factory.create()) {
var ispnProvider = session.getProvider(InfinispanConnectionProvider.class);
executor = ispnProvider.getExecutor("cluster-provider");
workCache = ispnProvider.getRemoteCache(WORK_CACHE_NAME);
clusterStartupTime = initClusterStartupTime(ispnProvider.getRemoteCache(WORK_CACHE_NAME), (int) (factory.getServerStartupTimestamp() / 1000));
notificationManager = new RemoteInfinispanNotificationManager(executor, ispnProvider.getRemoteCache(WORK_CACHE_NAME), getTopologyInfo(factory));
notificationManager.addClientListener();
logger.debugf("Provider initialized. Cluster startup time: %s", Time.toDate(clusterStartupTime));
lazyInit(session);
}
}
@ -82,10 +95,18 @@ public class RemoteInfinispanClusterProviderFactory implements ClusterProviderFa
return InfinispanUtils.isRemoteInfinispan();
}
private static TopologyInfo getTopologyInfo(KeycloakSessionFactory factory) {
try (var session = factory.create()) {
return session.getProvider(InfinispanConnectionProvider.class).getTopologyInfo();
private synchronized void lazyInit(KeycloakSession session) {
if (workCache != null) {
return;
}
var provider = session.getProvider(InfinispanConnectionProvider.class);
executor = provider.getExecutor("cluster-provider");
clusterStartupTime = initClusterStartupTime(provider.getRemoteCache(WORK_CACHE_NAME), (int) (session.getKeycloakSessionFactory().getServerStartupTimestamp() / 1000));
notificationManager = new RemoteInfinispanNotificationManager(executor, provider.getRemoteCache(WORK_CACHE_NAME), provider.getTopologyInfo());
notificationManager.addClientListener();
workCache = provider.getRemoteCache(WORK_CACHE_NAME);
logger.debugf("Provider initialized. Cluster startup time: %s", Time.toDate(clusterStartupTime));
}
private static int initClusterStartupTime(RemoteCache<String, Integer> cache, int serverStartupTime) {
@ -93,7 +114,6 @@ public class RemoteInfinispanClusterProviderFactory implements ClusterProviderFa
return clusterStartupTime == null ? serverStartupTime : clusterStartupTime;
}
static <V> V putIfAbsentWithRetries(RemoteCache<String, V> workCache, String key, V value, int taskTimeoutInSeconds) {
ByRef<V> ref = new ByRef<>(null);
@ -115,4 +135,24 @@ public class RemoteInfinispanClusterProviderFactory implements ClusterProviderFa
return ref.get();
}
@Override
public int clusterStartupTime() {
return clusterStartupTime;
}
@Override
public RemoteCache<String, LockEntry> cache() {
return workCache;
}
@Override
public RemoteInfinispanNotificationManager notificationManager() {
return notificationManager;
}
@Override
public Executor executor() {
return executor;
}
}

View file

@ -1,3 +1,20 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.cluster.infinispan.remote;
import java.lang.invoke.MethodHandles;

View file

@ -339,9 +339,9 @@ public class DefaultInfinispanConnectionProviderFactory implements InfinispanCon
defineClusteredCache(cacheManager, OFFLINE_USER_SESSION_CACHE_NAME, clusteredConfiguration);
defineClusteredCache(cacheManager, CLIENT_SESSION_CACHE_NAME, clusteredConfiguration);
defineClusteredCache(cacheManager, OFFLINE_CLIENT_SESSION_CACHE_NAME, clusteredConfiguration);
defineClusteredCache(cacheManager, LOGIN_FAILURE_CACHE_NAME, clusteredConfiguration);
if (InfinispanUtils.isEmbeddedInfinispan()) {
defineClusteredCache(cacheManager, LOGIN_FAILURE_CACHE_NAME, clusteredConfiguration);
defineClusteredCache(cacheManager, AUTHENTICATION_SESSIONS_CACHE_NAME, clusteredConfiguration);
var actionTokenBuilder = getActionTokenCacheConfig();

View file

@ -1,3 +1,20 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.connections.infinispan.remote;
import java.util.Arrays;

View file

@ -1,3 +1,20 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.connections.infinispan.remote;
import org.infinispan.client.hotrod.impl.InternalRemoteCache;

View file

@ -1,3 +1,20 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.infinispan.util;
import org.keycloak.common.Profile;

View file

@ -17,6 +17,11 @@
package org.keycloak.models.sessions.infinispan;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.concurrent.ConcurrentHashMap;
import org.infinispan.Cache;
import org.jboss.logging.Logger;
import org.keycloak.Config;
@ -34,21 +39,17 @@ import org.keycloak.models.sessions.infinispan.events.RealmRemovedSessionEvent;
import org.keycloak.models.sessions.infinispan.util.InfinispanKeyGenerator;
import org.keycloak.models.utils.KeycloakModelUtils;
import org.keycloak.models.utils.PostMigrationEvent;
import org.keycloak.provider.EnvironmentDependentProviderFactory;
import org.keycloak.provider.ProviderConfigProperty;
import org.keycloak.provider.ProviderConfigurationBuilder;
import org.keycloak.provider.ProviderEvent;
import org.keycloak.provider.ProviderEventListener;
import org.keycloak.sessions.AuthenticationSessionProviderFactory;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.concurrent.ConcurrentHashMap;
/**
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
*/
public class InfinispanAuthenticationSessionProviderFactory implements AuthenticationSessionProviderFactory<InfinispanAuthenticationSessionProvider> {
public class InfinispanAuthenticationSessionProviderFactory implements AuthenticationSessionProviderFactory<InfinispanAuthenticationSessionProvider>, EnvironmentDependentProviderFactory {
private static final Logger log = Logger.getLogger(InfinispanAuthenticationSessionProviderFactory.class);

View file

@ -18,6 +18,8 @@
package org.keycloak.models.sessions.infinispan;
import java.util.function.Supplier;
import org.infinispan.Cache;
import org.infinispan.client.hotrod.Flag;
import org.infinispan.client.hotrod.RemoteCache;
@ -31,13 +33,12 @@ import org.keycloak.models.KeycloakSession;
import org.keycloak.models.KeycloakSessionFactory;
import org.keycloak.models.SingleUseObjectProviderFactory;
import org.keycloak.models.sessions.infinispan.entities.SingleUseObjectValueEntity;
import java.util.function.Supplier;
import org.keycloak.provider.EnvironmentDependentProviderFactory;
/**
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
*/
public class InfinispanSingleUseObjectProviderFactory implements SingleUseObjectProviderFactory {
public class InfinispanSingleUseObjectProviderFactory implements SingleUseObjectProviderFactory, EnvironmentDependentProviderFactory {
private static final Logger LOG = Logger.getLogger(InfinispanSingleUseObjectProviderFactory.class);

View file

@ -17,22 +17,23 @@
package org.keycloak.models.sessions.infinispan;
import java.util.List;
import org.jboss.logging.Logger;
import org.keycloak.Config;
import org.keycloak.infinispan.util.InfinispanUtils;
import org.keycloak.models.KeycloakSession;
import org.keycloak.models.KeycloakSessionFactory;
import org.keycloak.provider.EnvironmentDependentProviderFactory;
import org.keycloak.provider.ProviderConfigProperty;
import org.keycloak.provider.ProviderConfigurationBuilder;
import org.keycloak.sessions.StickySessionEncoderProvider;
import org.keycloak.sessions.StickySessionEncoderProviderFactory;
import java.util.List;
/**
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
*/
public class InfinispanStickySessionEncoderProviderFactory implements StickySessionEncoderProviderFactory {
public class InfinispanStickySessionEncoderProviderFactory implements StickySessionEncoderProviderFactory, EnvironmentDependentProviderFactory {
private static final Logger log = Logger.getLogger(InfinispanStickySessionEncoderProviderFactory.class);

View file

@ -49,13 +49,14 @@ import org.keycloak.models.sessions.infinispan.remotestore.RemoteCacheSessionsLo
import org.keycloak.models.sessions.infinispan.util.SessionTimeouts;
import org.keycloak.models.utils.KeycloakModelUtils;
import org.keycloak.models.utils.PostMigrationEvent;
import org.keycloak.provider.EnvironmentDependentProviderFactory;
import java.util.Set;
/**
* @author <a href="mailto:mkanis@redhat.com">Martin Kanis</a>
*/
public class InfinispanUserLoginFailureProviderFactory implements UserLoginFailureProviderFactory {
public class InfinispanUserLoginFailureProviderFactory implements UserLoginFailureProviderFactory<InfinispanUserLoginFailureProvider>, EnvironmentDependentProviderFactory {
private static final Logger log = Logger.getLogger(InfinispanUserLoginFailureProviderFactory.class);
@ -69,7 +70,7 @@ public class InfinispanUserLoginFailureProviderFactory implements UserLoginFailu
SerializeExecutionsByKey<LoginFailureKey> serializer = new SerializeExecutionsByKey<>();
@Override
public UserLoginFailureProvider create(KeycloakSession session) {
public InfinispanUserLoginFailureProvider create(KeycloakSession session) {
InfinispanConnectionProvider connections = session.getProvider(InfinispanConnectionProvider.class);
Cache<LoginFailureKey, SessionEntityWrapper<LoginFailureEntity>> loginFailures = connections.getCache(InfinispanConnectionProvider.LOGIN_FAILURE_CACHE_NAME);
@ -90,14 +91,9 @@ public class InfinispanUserLoginFailureProviderFactory implements UserLoginFailu
KeycloakModelUtils.runJobInTransaction(factory, (KeycloakSession session) -> {
checkRemoteCaches(session);
registerClusterListeners(session);
// TODO [pruivo] to remove: workaround to run the testsuite.
if (InfinispanUtils.isEmbeddedInfinispan()) {
loadLoginFailuresFromRemoteCaches(session);
}
loadLoginFailuresFromRemoteCaches(session);
});
} else if (event instanceof UserModel.UserRemovedEvent) {
UserModel.UserRemovedEvent userRemovedEvent = (UserModel.UserRemovedEvent) event;
} else if (event instanceof UserModel.UserRemovedEvent userRemovedEvent) {
UserLoginFailureProvider provider = userRemovedEvent.getKeycloakSession().getProvider(UserLoginFailureProvider.class, getId());
provider.removeUserLoginFailure(userRemovedEvent.getRealm(), userRemovedEvent.getUser().getId());
}
@ -223,4 +219,9 @@ public class InfinispanUserLoginFailureProviderFactory implements UserLoginFailu
public int order() {
return InfinispanUtils.PROVIDER_ORDER;
}
@Override
public boolean isSupported(Config.Scope config) {
return InfinispanUtils.isEmbeddedInfinispan();
}
}

View file

@ -1,3 +1,20 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.models.sessions.infinispan;
/**

View file

@ -0,0 +1,207 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.models.sessions.infinispan.changes.remote;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.function.Predicate;
import io.reactivex.rxjava3.core.Completable;
import io.reactivex.rxjava3.core.Flowable;
import org.infinispan.client.hotrod.Flag;
import org.infinispan.client.hotrod.RemoteCache;
import org.infinispan.commons.util.concurrent.AggregateCompletionStage;
import org.infinispan.commons.util.concurrent.CompletableFutures;
import org.infinispan.commons.util.concurrent.CompletionStages;
import org.keycloak.models.AbstractKeycloakTransaction;
import org.keycloak.models.KeycloakSession;
import org.keycloak.models.sessions.infinispan.changes.remote.updater.Expiration;
import org.keycloak.models.sessions.infinispan.changes.remote.updater.Updater;
import org.keycloak.models.sessions.infinispan.changes.remote.updater.UpdaterFactory;
/**
* A {@link org.keycloak.models.KeycloakTransaction} implementation that keeps track of changes made to entities stored
* in a Infinispan cache.
*
* @param <K> The type of the Infinispan cache key.
* @param <V> The type of the Infinispan cache value.
* @param <T> The type of the {@link Updater} implementation.
*/
public class RemoteChangeLogTransaction<K, V, T extends Updater<K, V>> extends AbstractKeycloakTransaction {
private final Map<K, T> entityChanges;
private final UpdaterFactory<K, V, T> factory;
private final RemoteCache<K, V> cache;
private final KeycloakSession session;
private Predicate<V> removePredicate;
public RemoteChangeLogTransaction(UpdaterFactory<K, V, T> factory, RemoteCache<K, V> cache, KeycloakSession session) {
this.factory = Objects.requireNonNull(factory);
this.cache = Objects.requireNonNull(cache);
this.session = Objects.requireNonNull(session);
entityChanges = new ConcurrentHashMap<>(8);
}
@Override
protected void commitImpl() {
var stage = CompletionStages.aggregateCompletionStage();
doCommit(stage);
CompletionStages.join(stage.freeze());
entityChanges.clear();
removePredicate = null;
}
@Override
protected void rollbackImpl() {
entityChanges.clear();
removePredicate = null;
}
private void doCommit(AggregateCompletionStage<Void> stage) {
if (removePredicate != null) {
// TODO [pruivo] [optimization] with protostream, use delete by query: DELETE FROM ...
var rmStage = Flowable.fromPublisher(cache.publishEntriesWithMetadata(null, 2048))
.filter(e -> removePredicate.test(e.getValue().getValue()))
.map(Map.Entry::getKey)
.flatMapCompletable(this::removeKey)
.toCompletionStage(null);
stage.dependsOn(rmStage);
}
for (var updater : entityChanges.values()) {
if (updater.isReadOnly() || (removePredicate != null && removePredicate.test(updater.getValue()))) {
continue;
}
if (updater.isDeleted()) {
stage.dependsOn(cache.removeAsync(updater.getKey()));
continue;
}
var expiration = updater.computeExpiration(session);
if (expiration.isExpired()) {
stage.dependsOn(cache.removeAsync(updater.getKey()));
continue;
}
if (updater.isCreated()) {
stage.dependsOn(putIfAbsent(updater, expiration));
continue;
}
stage.dependsOn(replace(updater, expiration));
}
}
/**
* @return The {@link RemoteCache} tracked by the transaction.
*/
public RemoteCache<K, V> getCache() {
return cache;
}
/**
* Fetches the value associated to the {@code key}.
* <p>
* It fetches the value from the {@link RemoteCache} if a copy does not exist in the transaction.
*
* @param key The Infinispan cache key to fetch.
* @return The {@link Updater} to track further changes of the Infinispan cache value.
*/
public T get(K key) {
var updater = entityChanges.get(key);
if (updater != null) {
return updater;
}
var entity = cache.getWithMetadata(key);
if (entity == null) {
return null;
}
updater = factory.wrapFromCache(key, entity);
entityChanges.put(key, updater);
return updater.isDeleted() ? null : updater;
}
/**
* Tracks a new value to be created in the Infinispan cache.
*
* @param key The Infinispan cache key to be associated to the value.
* @param entity The Infinispan cache value.
* @return The {@link Updater} to track further changes of the Infinispan cache value.
*/
public T create(K key, V entity) {
var updater = factory.create(key, entity);
entityChanges.put(key, updater);
return updater;
}
/**
* Removes the {@code key} from the {@link RemoteCache}.
*
* @param key The Infinispan cache key to remove.
*/
public void remove(K key) {
var updater = entityChanges.get(key);
if (updater != null) {
updater.markDeleted();
return;
}
entityChanges.put(key, factory.deleted(key));
}
/**
* Removes all Infinispan cache values that satisfy the given predicate.
*
* @param predicate The {@link Predicate} which returns {@code true} for elements to be removed.
*/
public void removeIf(Predicate<V> predicate) {
if (removePredicate == null) {
removePredicate = predicate;
return;
}
removePredicate = removePredicate.or(predicate);
}
private CompletionStage<V> putIfAbsent(Updater<K, V> updater, Expiration expiration) {
return cache.withFlags(Flag.FORCE_RETURN_VALUE)
.putIfAbsentAsync(updater.getKey(), updater.getValue(), expiration.lifespan(), TimeUnit.MILLISECONDS, expiration.maxIdle(), TimeUnit.MILLISECONDS)
.thenApply(Objects::isNull)
.thenCompose(completed -> handleResponse(completed, updater, expiration));
}
private CompletionStage<V> replace(Updater<K, V> updater, Expiration expiration) {
return cache.replaceWithVersionAsync(updater.getKey(), updater.getValue(), updater.getVersionRead(), expiration.lifespan(), TimeUnit.MILLISECONDS, expiration.maxIdle(), TimeUnit.MILLISECONDS)
.thenCompose(completed -> handleResponse(completed, updater, expiration));
}
private CompletionStage<V> handleResponse(boolean completed, Updater<K, V> updater, Expiration expiration) {
return completed ? CompletableFutures.completedNull() : merge(updater, expiration);
}
private CompletionStage<V> merge(Updater<K, V> updater, Expiration expiration) {
return cache.computeAsync(updater.getKey(), updater, expiration.lifespan(), TimeUnit.MILLISECONDS, expiration.maxIdle(), TimeUnit.MILLISECONDS);
}
private Completable removeKey(K key) {
return Completable.fromCompletionStage(cache.removeAsync(key));
}
}

View file

@ -0,0 +1,115 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.models.sessions.infinispan.changes.remote.updater;
/**
* Base functionality of an {@link Updater} implementation.
* <p>
* It stores the Infinispan cache key, value, version, and it states. However, it does not keep track of the changed
* fields in the cache value, and it is the responsibility of the implementation to do that.
* <p>
* The method {@link #onFieldChanged()} must be invoked to track changes in the cache value.
*
* @param <K> The type of the Infinispan cache key.
* @param <V> The type of the Infinispan cache value.
*/
public abstract class BaseUpdater<K, V> implements Updater<K, V> {
private final K cacheKey;
private final V cacheValue;
private final long versionRead;
private UpdaterState state;
protected BaseUpdater(K cacheKey, V cacheValue, long versionRead, UpdaterState state) {
this.cacheKey = cacheKey;
this.cacheValue = cacheValue;
this.versionRead = versionRead;
this.state = state;
}
@Override
public final K getKey() {
return cacheKey;
}
@Override
public final V getValue() {
return cacheValue;
}
@Override
public final long getVersionRead() {
return versionRead;
}
@Override
public final boolean isDeleted() {
return state == UpdaterState.DELETED;
}
@Override
public final boolean isCreated() {
return state == UpdaterState.CREATED;
}
@Override
public final boolean isReadOnly() {
return state == UpdaterState.READ_ONLY;
}
@Override
public final void markDeleted() {
state = UpdaterState.DELETED;
}
/**
* Must be invoked when a field change to mark this updated and modified.
*/
protected final void onFieldChanged() {
state = state.stateAfterChange();
}
protected enum UpdaterState {
/**
* The cache value is created. It implies {@link #MODIFIED}.
*/
CREATED,
/**
* The cache value is deleted, and it will be removed from the Infinispan cache. It cannot be recreated.
*/
DELETED,
/**
* The cache value was read the Infinispan cache and was not modified.
*/
READ_ONLY {
@Override
UpdaterState stateAfterChange() {
return MODIFIED;
}
},
/**
* The cache value was read from the Infinispan cache and was modified. Changes will be merged into the current
* Infinispan cache value.
*/
MODIFIED;
UpdaterState stateAfterChange() {
return this;
}
}
}

View file

@ -0,0 +1,33 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.models.sessions.infinispan.changes.remote.updater;
import org.keycloak.models.sessions.infinispan.util.SessionTimeouts;
/**
* Expiration data for Infinispan storage, in milliseconds.
*
* @param maxIdle The entity max-idle. The entity will be removed if not accessed during this time.
* @param lifespan The entity lifespan. The entity will be removed after this time.
*/
public record Expiration(long maxIdle, long lifespan) {
public boolean isExpired() {
return maxIdle == SessionTimeouts.ENTRY_EXPIRED_FLAG || lifespan == SessionTimeouts.ENTRY_EXPIRED_FLAG;
}
}

View file

@ -0,0 +1,78 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.models.sessions.infinispan.changes.remote.updater;
import org.keycloak.models.KeycloakSession;
import org.keycloak.models.sessions.infinispan.changes.remote.RemoteChangeLogTransaction;
import java.util.function.BiFunction;
/**
* An interface used by {@link RemoteChangeLogTransaction}.
* <p>
* It keeps track of the changes made in the entity and applies them to the entity stored in Infinispan cache.
*
* @param <K> The Infinispan key type.
* @param <V> The Infinispan value type.
*/
public interface Updater<K, V> extends BiFunction<K, V, V> {
/**
* @return The Infinispan cache key.
*/
K getKey();
/**
* @return The up-to-date entity used by the transaction.
*/
V getValue();
/**
* @return The entity version when reading for the first time from Infinispan.
*/
long getVersionRead();
/**
* @return {@code true} if the entity was removed during the Keycloak transaction and it should be removed from
* Infinispan.
*/
boolean isDeleted();
/**
* @return {@code true} if the entity was created during the Keycloak transaction. Allows some optimization like
* put-if-absent.
*/
boolean isCreated();
/**
* @return {@code true} if the entity was not changed.
*/
boolean isReadOnly();
/**
* Marks the entity as deleted.
*/
void markDeleted();
/**
* Computes the expiration data for Infinispan cache.
*
* @param session The current Keycloak session.
* @return The {@link Expiration} data.
*/
Expiration computeExpiration(KeycloakSession session);
}

View file

@ -0,0 +1,56 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.models.sessions.infinispan.changes.remote.updater;
import org.infinispan.client.hotrod.MetadataValue;
/**
* A factory interface that creates, wraps or deletes entities.
*
* @param <K> The Infinispan key type.
* @param <V> The Infinispan value type.
* @param <T> The {@link Updater} concrete type.
*/
public interface UpdaterFactory<K, V, T extends Updater<K, V>> {
/**
* Creates an {@link Updater} for an entity created by the current Keycloak transaction.
*
* @param key The Infinispan key.
* @param entity The Infinispan value.
* @return The {@link Updater} to be used when updating the entity state.
*/
T create(K key, V entity);
/**
* Wraps an entity read from the Infinispan cache.
*
* @param key The Infinispan key.
* @param entity The Infinispan value.
* @return The {@link Updater} to be used when updating the entity state.
*/
T wrapFromCache(K key, MetadataValue<V> entity);
/**
* Deletes a entity that was not previous read by the Keycloak transaction.
*
* @param key The Infinispan key.
* @return The {@link Updater} for a deleted entity.
*/
T deleted(K key);
}

View file

@ -0,0 +1,160 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.models.sessions.infinispan.changes.remote.updater.loginfailures;
import org.infinispan.client.hotrod.MetadataValue;
import org.keycloak.models.KeycloakSession;
import org.keycloak.models.UserLoginFailureModel;
import org.keycloak.models.sessions.infinispan.changes.remote.updater.BaseUpdater;
import org.keycloak.models.sessions.infinispan.changes.remote.updater.Expiration;
import org.keycloak.models.sessions.infinispan.changes.remote.updater.Updater;
import org.keycloak.models.sessions.infinispan.entities.LoginFailureEntity;
import org.keycloak.models.sessions.infinispan.entities.LoginFailureKey;
import org.keycloak.models.sessions.infinispan.util.SessionTimeouts;
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
import java.util.function.Consumer;
/**
* Implementation of {@link Updater} and {@link UserLoginFailureModel}.
* <p>
* It keeps track of the changes made to the entity {@link LoginFailureEntity} and replays on commit.
*/
public class LoginFailuresUpdater extends BaseUpdater<LoginFailureKey, LoginFailureEntity> implements UserLoginFailureModel {
private final List<Consumer<LoginFailureEntity>> changes;
private LoginFailuresUpdater(LoginFailureKey key, LoginFailureEntity entity, long version, UpdaterState initialState) {
super(key, entity, version, initialState);
changes = new ArrayList<>(4);
}
public static LoginFailuresUpdater create(LoginFailureKey key, LoginFailureEntity entity) {
return new LoginFailuresUpdater(Objects.requireNonNull(key), Objects.requireNonNull(entity), -1, UpdaterState.CREATED);
}
public static LoginFailuresUpdater wrap(LoginFailureKey key, MetadataValue<LoginFailureEntity> entity) {
return new LoginFailuresUpdater(Objects.requireNonNull(key), Objects.requireNonNull(entity.getValue()), entity.getVersion(), UpdaterState.READ_ONLY);
}
public static LoginFailuresUpdater delete(LoginFailureKey key) {
return new LoginFailuresUpdater(Objects.requireNonNull(key), null, -1, UpdaterState.DELETED);
}
@Override
public Expiration computeExpiration(KeycloakSession session) {
var realm = session.realms().getRealm(getValue().getRealmId());
return new Expiration(
SessionTimeouts.getLoginFailuresMaxIdleMs(realm, null, getValue()),
SessionTimeouts.getLoginFailuresLifespanMs(realm, null, getValue()));
}
@Override
public LoginFailureEntity apply(LoginFailureKey ignored, LoginFailureEntity cachedEntity) {
assert !isDeleted();
assert !isReadOnly();
if (cachedEntity == null) {
//entity removed
return null;
}
changes.forEach(c -> c.accept(cachedEntity));
return cachedEntity;
}
@Override
public int getFailedLoginNotBefore() {
return getValue().getFailedLoginNotBefore();
}
@Override
public long getLastFailure() {
return getValue().getLastFailure();
}
@Override
public String getLastIPFailure() {
return getValue().getLastIPFailure();
}
@Override
public int getNumFailures() {
return getValue().getNumFailures();
}
@Override
public int getNumTemporaryLockouts() {
return getValue().getNumTemporaryLockouts();
}
@Override
public String getUserId() {
return getValue().getUserId();
}
@Override
public String getId() {
return getKey().toString();
}
@Override
public void clearFailures() {
addAndApplyChange(CLEAR);
}
@Override
public void setFailedLoginNotBefore(int notBefore) {
addAndApplyChange(e -> e.setFailedLoginNotBefore(notBefore));
}
@Override
public void incrementFailures() {
addAndApplyChange(INCREMENT_FAILURES);
}
@Override
public void incrementTemporaryLockouts() {
addAndApplyChange(INCREMENT_LOCK_OUTS);
}
@Override
public void setLastFailure(long lastFailure) {
addAndApplyChange(e -> e.setLastFailure(lastFailure));
}
@Override
public void setLastIPFailure(String ip) {
addAndApplyChange(e -> e.setLastIPFailure(ip));
}
private void addAndApplyChange(Consumer<LoginFailureEntity> change) {
if (change == CLEAR) {
changes.clear();
changes.add(CLEAR);
} else {
changes.add(change);
}
change.accept(getValue());
onFieldChanged();
}
private static final Consumer<LoginFailureEntity> CLEAR = LoginFailureEntity::clearFailures;
private static final Consumer<LoginFailureEntity> INCREMENT_FAILURES = e -> e.setNumFailures(e.getNumFailures() + 1);
private static final Consumer<LoginFailureEntity> INCREMENT_LOCK_OUTS = e -> e.setNumTemporaryLockouts(e.getNumTemporaryLockouts() + 1);
}

View file

@ -1,5 +1,27 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.models.sessions.infinispan.remote;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
import java.util.function.Predicate;
import org.keycloak.cluster.ClusterProvider;
import org.keycloak.common.util.Time;
import org.keycloak.models.ClientModel;
@ -16,10 +38,6 @@ import org.keycloak.sessions.AuthenticationSessionCompoundId;
import org.keycloak.sessions.AuthenticationSessionProvider;
import org.keycloak.sessions.RootAuthenticationSessionModel;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
public class RemoteInfinispanAuthenticationSessionProvider implements AuthenticationSessionProvider {
private final KeycloakSession session;
@ -78,15 +96,7 @@ public class RemoteInfinispanAuthenticationSessionProvider implements Authentica
@Override
public void onRealmRemoved(RealmModel realm) {
// TODO [pruivo] [optimization] with protostream, use delete by query: DELETE FROM ...
var cache = transaction.getCache();
try (var iterator = cache.retrieveEntries(null, 256)) {
while (iterator.hasNext()) {
var entry = iterator.next();
if (realm.getId().equals(((RootAuthenticationSessionEntity) entry.getValue()).getRealmId())) {
cache.removeAsync(entry.getKey());
}
}
}
transaction.removeIf(new RealmFilter(realm.getId()));
}
@Override
@ -132,4 +142,12 @@ public class RemoteInfinispanAuthenticationSessionProvider implements Authentica
transaction.remove(entity.getId());
}
}
private record RealmFilter(String realmId) implements Predicate<RootAuthenticationSessionEntity> {
@Override
public boolean test(RootAuthenticationSessionEntity entity) {
return Objects.equals(realmId, entity.getRealmId());
}
}
}

View file

@ -1,3 +1,20 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.models.sessions.infinispan.remote;
import java.lang.invoke.MethodHandles;
@ -11,6 +28,7 @@ import org.keycloak.models.KeycloakSession;
import org.keycloak.models.KeycloakSessionFactory;
import org.keycloak.models.sessions.infinispan.InfinispanAuthenticationSessionProviderFactory;
import org.keycloak.models.sessions.infinispan.entities.RootAuthenticationSessionEntity;
import org.keycloak.provider.EnvironmentDependentProviderFactory;
import org.keycloak.provider.ProviderConfigProperty;
import org.keycloak.provider.ProviderConfigurationBuilder;
import org.keycloak.sessions.AuthenticationSessionProviderFactory;
@ -19,7 +37,7 @@ import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.A
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.getRemoteCache;
import static org.keycloak.models.sessions.infinispan.InfinispanAuthenticationSessionProviderFactory.DEFAULT_AUTH_SESSIONS_LIMIT;
public class RemoteInfinispanAuthenticationSessionProviderFactory implements AuthenticationSessionProviderFactory<RemoteInfinispanAuthenticationSessionProvider> {
public class RemoteInfinispanAuthenticationSessionProviderFactory implements AuthenticationSessionProviderFactory<RemoteInfinispanAuthenticationSessionProvider>, EnvironmentDependentProviderFactory {
private final static Logger logger = Logger.getLogger(MethodHandles.lookup().lookupClass());

View file

@ -1,10 +1,21 @@
package org.keycloak.models.sessions.infinispan.remote;
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.
*/
import org.infinispan.client.hotrod.RemoteCache;
import org.infinispan.commons.util.concurrent.AggregateCompletionStage;
import org.infinispan.commons.util.concurrent.CompletionStages;
import org.jboss.logging.Logger;
import org.keycloak.models.KeycloakTransaction;
package org.keycloak.models.sessions.infinispan.remote;
import java.lang.invoke.MethodHandles;
import java.util.LinkedHashMap;
@ -12,67 +23,53 @@ import java.util.Map;
import java.util.Objects;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.function.Predicate;
public class RemoteInfinispanKeycloakTransaction<K, V> implements KeycloakTransaction {
import io.reactivex.rxjava3.core.Completable;
import io.reactivex.rxjava3.core.Flowable;
import org.infinispan.client.hotrod.MetadataValue;
import org.infinispan.client.hotrod.RemoteCache;
import org.infinispan.commons.util.concurrent.AggregateCompletionStage;
import org.infinispan.commons.util.concurrent.CompletionStages;
import org.jboss.logging.Logger;
import org.keycloak.models.AbstractKeycloakTransaction;
public class RemoteInfinispanKeycloakTransaction<K, V> extends AbstractKeycloakTransaction {
private final static Logger logger = Logger.getLogger(MethodHandles.lookup().lookupClass());
private boolean active;
private boolean rollback;
private final Map<K, Operation<K, V>> tasks = new LinkedHashMap<>();
private final RemoteCache<K, V> cache;
private Predicate<V> removePredicate;
public RemoteInfinispanKeycloakTransaction(RemoteCache<K, V> cache) {
this.cache = Objects.requireNonNull(cache);
}
@Override
public void begin() {
active = true;
tasks.clear();
}
@Override
public void commit() {
active = false;
if (rollback) {
throw new RuntimeException("Rollback only!");
}
protected void commitImpl() {
AggregateCompletionStage<Void> stage = CompletionStages.aggregateCompletionStage();
if (removePredicate != null) {
// TODO [pruivo] [optimization] with protostream, use delete by query: DELETE FROM ...
var rmStage = Flowable.fromPublisher(cache.publishEntriesWithMetadata(null, 2048))
.filter(this::shouldRemoveEntry)
.map(Map.Entry::getKey)
.flatMapCompletable(this::removeKey)
.toCompletionStage(null);
stage.dependsOn(rmStage);
}
tasks.values().stream()
.filter(this::shouldCommitOperation)
.map(this::commitOperation)
.forEach(stage::dependsOn);
try {
CompletionStages.await(stage.freeze());
} catch (ExecutionException e) {
throw new RuntimeException(e);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RuntimeException(e);
}
}
@Override
public void rollback() {
active = false;
CompletionStages.join(stage.freeze());
tasks.clear();
}
@Override
public void setRollbackOnly() {
rollback = true;
}
@Override
public boolean getRollbackOnly() {
return rollback;
}
@Override
public boolean isActive() {
return active;
protected void rollbackImpl() {
tasks.clear();
}
public void put(K key, V value, long lifespan, TimeUnit timeUnit) {
@ -126,6 +123,39 @@ public class RemoteInfinispanKeycloakTransaction<K, V> implements KeycloakTransa
return cache;
}
/**
* Removes all Infinispan cache values that satisfy the given predicate.
*
* @param predicate The {@link Predicate} which returns {@code true} for elements to be removed.
*/
public void removeIf(Predicate<V> predicate) {
if (removePredicate == null) {
removePredicate = predicate;
return;
}
removePredicate = removePredicate.or(predicate);
}
private Completable removeKey(K key) {
return Completable.fromCompletionStage(cache.removeAsync(key));
}
private boolean shouldCommitOperation(Operation<K, V> operation) {
// Commit if any:
// 1. it is a removal operation (no value to test the predicate).
// 2. remove predicate is not present.
// 3. value does not match the remove predicate.
return !operation.hasValue() ||
removePredicate == null ||
!removePredicate.test(operation.getValue());
}
private boolean shouldRemoveEntry(Map.Entry<K, MetadataValue<V>> entry) {
// invoked by stream, so removePredicate is not null
assert removePredicate != null;
return removePredicate.test(entry.getValue().getValue());
}
private CompletionStage<?> commitOperation(Operation<K, V> operation) {
try {
return operation.execute(cache);

View file

@ -1,3 +1,20 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.models.sessions.infinispan.remote;
import org.infinispan.client.hotrod.Flag;

View file

@ -1,5 +1,24 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.models.sessions.infinispan.remote;
import java.lang.invoke.MethodHandles;
import org.infinispan.client.hotrod.RemoteCache;
import org.jboss.logging.Logger;
import org.keycloak.Config;
@ -8,13 +27,12 @@ import org.keycloak.models.KeycloakSession;
import org.keycloak.models.KeycloakSessionFactory;
import org.keycloak.models.SingleUseObjectProviderFactory;
import org.keycloak.models.sessions.infinispan.entities.SingleUseObjectValueEntity;
import java.lang.invoke.MethodHandles;
import org.keycloak.provider.EnvironmentDependentProviderFactory;
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.ACTION_TOKEN_CACHE;
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.getRemoteCache;
public class RemoteInfinispanSingleUseObjectProviderFactory implements SingleUseObjectProviderFactory<RemoteInfinispanSingleUseObjectProvider> {
public class RemoteInfinispanSingleUseObjectProviderFactory implements SingleUseObjectProviderFactory<RemoteInfinispanSingleUseObjectProvider>, EnvironmentDependentProviderFactory {
private final static Logger logger = Logger.getLogger(MethodHandles.lookup().lookupClass());

View file

@ -1,20 +1,38 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.models.sessions.infinispan.remote;
import java.lang.invoke.MethodHandles;
import java.util.List;
import org.jboss.logging.Logger;
import org.keycloak.Config;
import org.keycloak.connections.infinispan.InfinispanUtil;
import org.keycloak.infinispan.util.InfinispanUtils;
import org.keycloak.models.KeycloakSession;
import org.keycloak.models.KeycloakSessionFactory;
import org.keycloak.provider.EnvironmentDependentProviderFactory;
import org.keycloak.provider.ProviderConfigProperty;
import org.keycloak.provider.ProviderConfigurationBuilder;
import org.keycloak.sessions.StickySessionEncoderProvider;
import org.keycloak.sessions.StickySessionEncoderProviderFactory;
import java.lang.invoke.MethodHandles;
import java.util.List;
public class RemoteStickySessionEncoderProviderFactory implements StickySessionEncoderProviderFactory {
public class RemoteStickySessionEncoderProviderFactory implements StickySessionEncoderProviderFactory, EnvironmentDependentProviderFactory {
private static final Logger log = Logger.getLogger(MethodHandles.lookup().lookupClass());
private static final char SEPARATOR = '.';

View file

@ -0,0 +1,85 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.models.sessions.infinispan.remote;
import java.lang.invoke.MethodHandles;
import java.util.Objects;
import org.jboss.logging.Logger;
import org.keycloak.models.RealmModel;
import org.keycloak.models.UserLoginFailureModel;
import org.keycloak.models.UserLoginFailureProvider;
import org.keycloak.models.sessions.infinispan.changes.remote.RemoteChangeLogTransaction;
import org.keycloak.models.sessions.infinispan.changes.remote.updater.loginfailures.LoginFailuresUpdater;
import org.keycloak.models.sessions.infinispan.entities.LoginFailureEntity;
import org.keycloak.models.sessions.infinispan.entities.LoginFailureKey;
import static org.keycloak.common.util.StackUtil.getShortStackTrace;
public class RemoteUserLoginFailureProvider implements UserLoginFailureProvider {
private static final Logger log = Logger.getLogger(MethodHandles.lookup().lookupClass());
private final RemoteChangeLogTransaction<LoginFailureKey, LoginFailureEntity, LoginFailuresUpdater> transaction;
public RemoteUserLoginFailureProvider(RemoteChangeLogTransaction<LoginFailureKey, LoginFailureEntity, LoginFailuresUpdater> transaction) {
this.transaction = Objects.requireNonNull(transaction);
}
@Override
public UserLoginFailureModel getUserLoginFailure(RealmModel realm, String userId) {
if (log.isTraceEnabled()) {
log.tracef("getUserLoginFailure(%s, %s)%s", realm, userId, getShortStackTrace());
}
return transaction.get(new LoginFailureKey(realm.getId(), userId));
}
@Override
public UserLoginFailureModel addUserLoginFailure(RealmModel realm, String userId) {
if (log.isTraceEnabled()) {
log.tracef("addUserLoginFailure(%s, %s)%s", realm, userId, getShortStackTrace());
}
var key = new LoginFailureKey(realm.getId(), userId);
var entity = new LoginFailureEntity(realm.getId(), userId);
return transaction.create(key, entity);
}
@Override
public void removeUserLoginFailure(RealmModel realm, String userId) {
if (log.isTraceEnabled()) {
log.tracef("removeUserLoginFailure(%s, %s)%s", realm, userId, getShortStackTrace());
}
transaction.remove(new LoginFailureKey(realm.getId(), userId));
}
@Override
public void removeAllUserLoginFailures(RealmModel realm) {
if (log.isTraceEnabled()) {
log.tracef("removeAllUserLoginFailures(%s)%s", realm, getShortStackTrace());
}
transaction.removeIf(entity -> Objects.equals(entity.getRealmId(), realm.getId()));
}
@Override
public void close() {
}
}

View file

@ -0,0 +1,105 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.models.sessions.infinispan.remote;
import org.infinispan.client.hotrod.MetadataValue;
import org.infinispan.client.hotrod.RemoteCache;
import org.jboss.logging.Logger;
import org.keycloak.Config;
import org.keycloak.infinispan.util.InfinispanUtils;
import org.keycloak.models.KeycloakSession;
import org.keycloak.models.KeycloakSessionFactory;
import org.keycloak.models.UserLoginFailureProvider;
import org.keycloak.models.UserLoginFailureProviderFactory;
import org.keycloak.models.UserModel;
import org.keycloak.models.sessions.infinispan.changes.remote.RemoteChangeLogTransaction;
import org.keycloak.models.sessions.infinispan.changes.remote.updater.UpdaterFactory;
import org.keycloak.models.sessions.infinispan.changes.remote.updater.loginfailures.LoginFailuresUpdater;
import org.keycloak.models.sessions.infinispan.entities.LoginFailureEntity;
import org.keycloak.models.sessions.infinispan.entities.LoginFailureKey;
import org.keycloak.provider.EnvironmentDependentProviderFactory;
import java.lang.invoke.MethodHandles;
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.LOGIN_FAILURE_CACHE_NAME;
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.getRemoteCache;
public class RemoteUserLoginFailureProviderFactory implements UserLoginFailureProviderFactory<RemoteUserLoginFailureProvider>, UpdaterFactory<LoginFailureKey, LoginFailureEntity, LoginFailuresUpdater>, EnvironmentDependentProviderFactory {
private static final Logger log = Logger.getLogger(MethodHandles.lookup().lookupClass());
private volatile RemoteCache<LoginFailureKey, LoginFailureEntity> cache;
@Override
public RemoteUserLoginFailureProvider create(KeycloakSession session) {
var tx = new RemoteChangeLogTransaction<>(this, cache, session);
session.getTransactionManager().enlistAfterCompletion(tx);
return new RemoteUserLoginFailureProvider(tx);
}
@Override
public void init(Config.Scope config) {
}
@Override
public void postInit(final KeycloakSessionFactory factory) {
cache = getRemoteCache(factory, LOGIN_FAILURE_CACHE_NAME);
factory.register(event -> {
if (event instanceof UserModel.UserRemovedEvent userRemovedEvent) {
UserLoginFailureProvider provider = userRemovedEvent.getKeycloakSession().getProvider(UserLoginFailureProvider.class, getId());
provider.removeUserLoginFailure(userRemovedEvent.getRealm(), userRemovedEvent.getUser().getId());
}
});
log.debugf("Post Init. Cache=%s", cache.getName());
}
@Override
public void close() {
cache = null;
}
@Override
public String getId() {
return InfinispanUtils.REMOTE_PROVIDER_ID;
}
@Override
public int order() {
return InfinispanUtils.PROVIDER_ORDER;
}
@Override
public boolean isSupported(Config.Scope config) {
return InfinispanUtils.isRemoteInfinispan();
}
@Override
public LoginFailuresUpdater create(LoginFailureKey key, LoginFailureEntity entity) {
return LoginFailuresUpdater.create(key, entity);
}
@Override
public LoginFailuresUpdater wrapFromCache(LoginFailureKey key, MetadataValue<LoginFailureEntity> entity) {
assert entity != null;
return LoginFailuresUpdater.wrap(key, entity);
}
@Override
public LoginFailuresUpdater deleted(LoginFailureKey key) {
return LoginFailuresUpdater.delete(key);
}
}

View file

@ -35,7 +35,9 @@ public class SessionTimeouts {
/**
* This indicates that entry is already expired and should be removed from the cache
*/
public static final long ENTRY_EXPIRED_FLAG = -2l;
public static final long ENTRY_EXPIRED_FLAG = -2;
private static final long IMMORTAL_FLAG = -1;
/**
* Get the maximum lifespan, which this userSession can remain in the infinispan cache.
@ -211,7 +213,7 @@ public class SessionTimeouts {
* @return
*/
public static long getLoginFailuresLifespanMs(RealmModel realm, ClientModel client, LoginFailureEntity loginFailureEntity) {
return -1l;
return IMMORTAL_FLAG;
}
@ -224,6 +226,6 @@ public class SessionTimeouts {
* @return
*/
public static long getLoginFailuresMaxIdleMs(RealmModel realm, ClientModel client, LoginFailureEntity loginFailureEntity) {
return -1l;
return IMMORTAL_FLAG;
}
}

View file

@ -15,4 +15,5 @@
# limitations under the License.
#
org.keycloak.models.sessions.infinispan.InfinispanUserLoginFailureProviderFactory
org.keycloak.models.sessions.infinispan.InfinispanUserLoginFailureProviderFactory
org.keycloak.models.sessions.infinispan.remote.RemoteUserLoginFailureProviderFactory

View file

@ -17,11 +17,10 @@
package org.keycloak.cluster;
import org.keycloak.provider.EnvironmentDependentProviderFactory;
import org.keycloak.provider.ProviderFactory;
/**
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
*/
public interface ClusterProviderFactory extends ProviderFactory<ClusterProvider>, EnvironmentDependentProviderFactory {
public interface ClusterProviderFactory extends ProviderFactory<ClusterProvider> {
}

View file

@ -69,6 +69,7 @@ import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.A
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.AUTHENTICATION_SESSIONS_CACHE_NAME;
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.CLIENT_SESSION_CACHE_NAME;
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.CLUSTERED_CACHE_NAMES;
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.LOGIN_FAILURE_CACHE_NAME;
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.OFFLINE_CLIENT_SESSION_CACHE_NAME;
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.OFFLINE_USER_SESSION_CACHE_NAME;
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.USER_SESSION_CACHE_NAME;
@ -125,6 +126,7 @@ public class CacheManagerFactory {
}
private RemoteCacheManager startRemoteCacheManager() {
logger.info("Starting Infinispan remote cache manager (Hot Rod Client)");
String cacheRemoteHost = requiredStringProperty(CACHE_REMOTE_HOST_PROPERTY);
Integer cacheRemotePort = Configuration.getOptionalKcValue(CACHE_REMOTE_PORT_PROPERTY)
.map(Integer::parseInt)
@ -172,6 +174,7 @@ public class CacheManagerFactory {
}
private CompletableFuture<DefaultCacheManager> startEmbeddedCacheManager(String config) {
logger.info("Starting Infinispan embedded cache manager");
ConfigurationBuilderHolder builder = new ParserRegistry().parse(config);
if (builder.getNamedConfigurationBuilders().entrySet().stream().anyMatch(c -> c.getValue().clustering().cacheMode().isClustered())) {
@ -225,6 +228,7 @@ public class CacheManagerFactory {
builders.remove(WORK_CACHE_NAME);
builders.remove(AUTHENTICATION_SESSIONS_CACHE_NAME);
builders.remove(ACTION_TOKEN_CACHE);
builders.remove(LOGIN_FAILURE_CACHE_NAME);
}
var start = isStartEagerly();
@ -232,7 +236,7 @@ public class CacheManagerFactory {
}
private static boolean isRemoteTLSEnabled() {
return Boolean.parseBoolean(System.getProperty("kc.cache-remote-tls-enabled", Boolean.TRUE.toString()));
return Configuration.isTrue(CachingOptions.CACHE_REMOTE_TLS_ENABLED);
}
private static boolean isRemoteAuthenticationEnabled() {
@ -241,7 +245,7 @@ public class CacheManagerFactory {
}
private static boolean createRemoteCaches() {
return Boolean.parseBoolean(System.getProperty("kc.cache-remote-create-caches", Boolean.FALSE.toString()));
return Boolean.getBoolean("kc.cache-remote-create-caches");
}
private static SSLContext createSSLContext() {

View file

@ -17,11 +17,10 @@
package org.keycloak.models;
import org.keycloak.provider.EnvironmentDependentProviderFactory;
import org.keycloak.provider.ProviderFactory;
/**
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
*/
public interface SingleUseObjectProviderFactory<T extends SingleUseObjectProvider> extends ProviderFactory<T>, EnvironmentDependentProviderFactory {
public interface SingleUseObjectProviderFactory<T extends SingleUseObjectProvider> extends ProviderFactory<T> {
}

View file

@ -17,11 +17,10 @@
package org.keycloak.sessions;
import org.keycloak.provider.EnvironmentDependentProviderFactory;
import org.keycloak.provider.ProviderFactory;
/**
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
*/
public interface AuthenticationSessionProviderFactory<T extends AuthenticationSessionProvider> extends ProviderFactory<T>, EnvironmentDependentProviderFactory {
public interface AuthenticationSessionProviderFactory<T extends AuthenticationSessionProvider> extends ProviderFactory<T> {
}

View file

@ -17,13 +17,12 @@
package org.keycloak.sessions;
import org.keycloak.provider.EnvironmentDependentProviderFactory;
import org.keycloak.provider.ProviderFactory;
/**
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
*/
public interface StickySessionEncoderProviderFactory extends ProviderFactory<StickySessionEncoderProvider>, EnvironmentDependentProviderFactory {
public interface StickySessionEncoderProviderFactory extends ProviderFactory<StickySessionEncoderProvider> {
/**
* For testing purpose only

View file

@ -66,6 +66,7 @@
<migration.73.version>4.8.3.Final</migration.73.version>
<!-- By default, skip docker-maven-plugin when running base tests-->
<docker.infinispan.skip>true</docker.infinispan.skip>
<docker.database.skip>true</docker.database.skip>
<docker.database.postStart>/bin/true</docker.database.postStart>
<docker.database.wait-for-log-regex>NEVER-MATCHING-REGEX</docker.database.wait-for-log-regex>

View file

@ -384,9 +384,6 @@
<groupId>io.fabric8</groupId>
<artifactId>docker-maven-plugin</artifactId>
<version>${docker.maven.plugin.version}</version>
<configuration>
<skip>${docker.database.skip}</skip>
</configuration>
<executions>
<execution>
<id>start-db-container</id>
@ -395,6 +392,7 @@
<goal>start</goal>
</goals>
<configuration>
<skip>${docker.database.skip}</skip>
<showLogs>true</showLogs>
<images>
<image>
@ -447,6 +445,9 @@
<goals>
<goal>stop</goal>
</goals>
<configuration>
<skip>${docker.database.skip}</skip>
</configuration>
</execution>
</executions>
</plugin>
@ -765,6 +766,71 @@
</dependencies>
</profile>
<profile>
<id>infinispan-server</id>
<properties>
<docker.infinispan.skip>false</docker.infinispan.skip>
</properties>
<build>
<plugins>
<plugin>
<groupId>io.fabric8</groupId>
<artifactId>docker-maven-plugin</artifactId>
<version>${docker.maven.plugin.version}</version>
<executions>
<execution>
<id>start-ispn-container</id>
<phase>process-test-classes</phase>
<goals>
<goal>start</goal>
</goals>
<configuration>
<skip>${docker.infinispan.skip}</skip>
<showLogs>true</showLogs>
<images>
<image>
<alias>infinispan</alias>
<name>quay.io/infinispan/server:${infinispan.version}</name>
<run>
<ports>
<port>11222:11222</port>
</ports>
<env>
<USER>keycloak</USER>
<PASS>Password1!</PASS>
</env>
<net>host</net>
<cmd>-b 127.0.0.1 -k 127.0.0.1</cmd>
<wait>
<log>.*ISPN080001.*</log>
<time>2000000</time>
<kill>10000</kill>
</wait>
</run>
</image>
</images>
</configuration>
</execution>
<execution>
<id>stop-ispn-container</id>
<phase>test</phase>
<goals>
<goal>stop</goal>
</goals>
<configuration>
<skip>${docker.infinispan.skip}</skip>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<artifactId>maven-compiler-plugin</artifactId>
<version>${maven-compiler-plugin.version}</version>
</plugin>
</plugins>
</build>
</profile>
</profiles>
</project>

View file

@ -29,6 +29,8 @@ import java.security.KeyManagementException;
import java.security.NoSuchAlgorithmException;
import java.security.cert.X509Certificate;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
@ -37,6 +39,8 @@ import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import javax.net.ssl.HostnameVerifier;
import javax.net.ssl.HttpsURLConnection;
import javax.net.ssl.SSLContext;
@ -207,6 +211,16 @@ public abstract class AbstractQuarkusDeployableContainer implements DeployableCo
addStorageOptions(storeProvider, commands);
addFeaturesOption(commands);
var features = getDefaultFeatures();
if (features.contains("remote-cache") && features.contains("multi-site")) {
commands.add("--cache-remote-host=localhost");
commands.add("--cache-remote-username=keycloak");
commands.add("--cache-remote-password=Password1!");
commands.add("--cache-remote-tls-enabled=false");
commands.add("--spi-connections-infinispan-quarkus-site-name=test");
configuration.appendJavaOpts("-Dkc.cache-remote-create-caches=true");
}
return commands;
}
@ -407,4 +421,12 @@ public abstract class AbstractQuarkusDeployableContainer implements DeployableCo
configuration.appendJavaOpts("-Djava.security.properties=" + System.getProperty("auth.server.java.security.file"));
}
private Collection<String> getDefaultFeatures() {
var features = configuration.getDefaultFeatures();
if (features == null || features.isBlank()) {
return List.of();
}
return Arrays.stream(features.split(",")).collect(Collectors.toSet());
}
}

View file

@ -22,6 +22,7 @@ import org.jboss.resteasy.client.jaxrs.ResteasyClient;
import org.jboss.resteasy.client.jaxrs.ResteasyClientBuilder;
import org.jboss.resteasy.client.jaxrs.ResteasyWebTarget;
import org.junit.Assert;
import org.junit.AssumptionViolatedException;
import org.keycloak.common.Profile;
import org.keycloak.testsuite.ProfileAssume;
import org.keycloak.testsuite.client.resources.TestApplicationResource;
@ -185,6 +186,8 @@ public class KeycloakTestingClient implements AutoCloseable {
if (t instanceof AssertionError) {
throw (AssertionError) t;
} else if (t instanceof AssumptionViolatedException) {
throw (AssumptionViolatedException) t;
} else {
throw new RunOnServerException(t);
}

View file

@ -78,6 +78,7 @@ import java.util.Calendar;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Scanner;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
@ -86,7 +87,9 @@ import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.Consumer;
import java.util.function.Supplier;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
@ -159,6 +162,7 @@ public abstract class AbstractKeycloakTest {
@Before
public void beforeAbstractKeycloakTest() throws Exception {
ProfileAssume.setTestContext(testContext);
adminClient = testContext.getAdminClient();
if (adminClient == null || adminClient.isClosed()) {
reconnectAdminClient();
@ -764,4 +768,31 @@ public abstract class AbstractKeycloakTest {
}
}
public static <T> void eventuallyEquals(String message, T expected, Supplier<T> actual) {
eventuallyEquals(message, expected, actual, 10000, 100, MILLISECONDS);
}
public static <T> void eventuallyEquals(String message, T expected, Supplier<T> actual, long timeout,
long pollInterval, TimeUnit unit) {
if (pollInterval <= 0) {
throw new IllegalArgumentException("Check interval must be positive");
}
try {
long expectedEndTime = System.nanoTime() + TimeUnit.NANOSECONDS.convert(timeout, unit);
long sleepMillis = MILLISECONDS.convert(pollInterval, unit);
do {
if (Objects.equals(expected, actual.get())) {
return;
}
Thread.sleep(sleepMillis);
} while (expectedEndTime - System.nanoTime() > 0);
//last attempt
assertEquals(message, expected, actual.get());
} catch (Exception e) {
throw new RuntimeException("Unexpected!", e);
}
}
}

View file

@ -17,12 +17,16 @@
package org.keycloak.testsuite.model;
import java.util.concurrent.atomic.AtomicReference;
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.keycloak.common.Profile;
import org.keycloak.common.util.Time;
import org.keycloak.models.ClientModel;
import org.keycloak.models.Constants;
import org.keycloak.models.KeycloakSession;
import org.keycloak.models.RealmModel;
import org.keycloak.models.UserManager;
@ -37,15 +41,13 @@ import org.keycloak.sessions.CommonClientSessionModel;
import org.keycloak.sessions.RootAuthenticationSessionModel;
import org.keycloak.testsuite.AbstractTestRealmKeycloakTest;
import org.keycloak.testsuite.arquillian.annotation.ModelTest;
import org.keycloak.testsuite.util.InfinispanTestTimeServiceRule;
import java.util.concurrent.atomic.AtomicReference;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.core.Is.is;
import static org.hamcrest.core.IsNull.notNullValue;
import static org.hamcrest.core.IsNull.nullValue;
import static org.hamcrest.MatcherAssert.assertThat;
import org.keycloak.models.Constants;
import org.keycloak.testsuite.util.InfinispanTestTimeServiceRule;
import static org.junit.Assume.assumeFalse;
/**
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
@ -212,10 +214,10 @@ public class AuthenticationSessionProviderTest extends AbstractTestRealmKeycloak
@Test
@ModelTest
public void testExpiredAuthSessions(KeycloakSession session) {
assumeFalse(Profile.isFeatureEnabled(Profile.Feature.REMOTE_CACHE));
AtomicReference<String> authSessionID = new AtomicReference<>();
KeycloakModelUtils.runJobInTransaction(session.getKeycloakSessionFactory(), (KeycloakSession sessionExpired) -> {
KeycloakSession mainSession = sessionExpired;
KeycloakModelUtils.runJobInTransaction(session.getKeycloakSessionFactory(), mainSession -> {
try {
// AccessCodeLifespan = 10 ; AccessCodeLifespanUserAction = 10 ; AccessCodeLifespanLogin = 30
setAccessCodeLifespan(mainSession, 10, 10, 30);

View file

@ -23,6 +23,7 @@ import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.keycloak.common.util.Time;
import org.keycloak.infinispan.util.InfinispanUtils;
import org.keycloak.models.AuthenticatedClientSessionModel;
import org.keycloak.models.ClientModel;
import org.keycloak.models.KeycloakSession;
@ -322,14 +323,20 @@ public class UserSessionProviderTest extends AbstractTestRealmKeycloakTest {
RealmModel realm = session.realms().getRealmByName("test");
createSessions(session);
KeycloakModelUtils.runJobInTransaction(session.getKeycloakSessionFactory(), (KeycloakSession kcSession) -> {
kcSession.sessions().removeUserSessions(realm);
});
KeycloakModelUtils.runJobInTransaction(session.getKeycloakSessionFactory(), kcSession -> kcSession.sessions().removeUserSessions(realm));
assertEquals(0, session.sessions().getUserSessionsStream(realm, session.users().getUserByUsername(realm, "user1"))
.count());
assertEquals(0, session.sessions().getUserSessionsStream(realm, session.users().getUserByUsername(realm, "user2"))
.count());
var user1 = session.users().getUserByUsername(realm, "user1");
var user2 = session.users().getUserByUsername(realm, "user2");
// TODO! [pruivo] to be removed when the session cache is remote only
// TODO! the Hot Rod events are async
if (InfinispanUtils.isRemoteInfinispan()) {
eventuallyEquals(null, 0L, () -> session.sessions().getUserSessionsStream(realm, user1).count());
eventuallyEquals(null, 0L, () -> session.sessions().getUserSessionsStream(realm, user2).count());
} else {
assertEquals(0, session.sessions().getUserSessionsStream(realm, user1).count());
assertEquals(0, session.sessions().getUserSessionsStream(realm, user2).count());
}
}
@Test

View file

@ -28,6 +28,7 @@ import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.http.impl.client.HttpClientBuilder;
import org.apache.http.message.BasicNameValuePair;
import org.junit.Assert;
import org.junit.Assume;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
@ -37,6 +38,7 @@ import org.keycloak.admin.client.resource.ClientScopeResource;
import org.keycloak.admin.client.resource.ClientsResource;
import org.keycloak.admin.client.resource.RealmResource;
import org.keycloak.admin.client.resource.UserResource;
import org.keycloak.common.Profile;
import org.keycloak.common.enums.SslRequired;
import org.keycloak.common.util.Base64Url;
import org.keycloak.crypto.Algorithm;
@ -44,6 +46,7 @@ import org.keycloak.crypto.ECDSAAlgorithm;
import org.keycloak.crypto.KeyUse;
import org.keycloak.events.Details;
import org.keycloak.events.Errors;
import org.keycloak.infinispan.util.InfinispanUtils;
import org.keycloak.jose.jwk.JWK;
import org.keycloak.jose.jws.JWSHeader;
import org.keycloak.jose.jws.JWSInput;
@ -69,6 +72,7 @@ import org.keycloak.representations.idm.UserRepresentation;
import org.keycloak.testsuite.AbstractKeycloakTest;
import org.keycloak.testsuite.ActionURIUtils;
import org.keycloak.testsuite.AssertEvents;
import org.keycloak.testsuite.ProfileAssume;
import org.keycloak.testsuite.admin.ApiUtil;
import org.keycloak.testsuite.util.AdminClientUtil;
import org.keycloak.testsuite.util.ClientBuilder;
@ -110,6 +114,7 @@ import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assume.*;
import static org.keycloak.testsuite.Assert.assertExpiration;
import static org.keycloak.testsuite.admin.AbstractAdminTest.loadJson;
import static org.keycloak.testsuite.admin.ApiUtil.findClientByClientId;
@ -383,6 +388,7 @@ public class AccessTokenTest extends AbstractKeycloakTest {
@Test
public void accessTokenCodeExpired() {
ProfileAssume.assumeFeatureDisabled(Profile.Feature.REMOTE_CACHE);
getTestingClient().testing().setTestingInfinispanTimeService();
RealmManager.realm(adminClient.realm("test")).accessCodeLifeSpan(1);
oauth.doLogin("test-user@localhost", "password");

View file

@ -15,9 +15,9 @@
<packaging>jar</packaging>
<properties>
<maven.compiler.release>11</maven.compiler.release>
<maven.compiler.source>11</maven.compiler.source>
<maven.compiler.target>11</maven.compiler.target>
<maven.compiler.release>17</maven.compiler.release>
<maven.compiler.source>17</maven.compiler.source>
<maven.compiler.target>17</maven.compiler.target>
<keycloak.connectionsJpa.driver>org.h2.Driver</keycloak.connectionsJpa.driver>
<keycloak.connectionsJpa.database>keycloak</keycloak.connectionsJpa.database>
@ -224,6 +224,19 @@
<properties>
<keycloak.model.parameters>CrossDCInfinispan,Jpa,PersistentUserSessions</keycloak.model.parameters>
</properties>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<systemPropertyVariables>
<keycloak.profile.feature.multi_site>enabled</keycloak.profile.feature.multi_site>
</systemPropertyVariables>
</configuration>
</plugin>
</plugins>
</build>
</profile>
<profile>

View file

@ -1,3 +1,20 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.testsuite.model.infinispan;
import java.util.Arrays;
@ -22,6 +39,7 @@ import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.A
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.AUTHENTICATION_SESSIONS_CACHE_NAME;
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.CLUSTERED_CACHE_NAMES;
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.LOCAL_CACHE_NAMES;
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.LOGIN_FAILURE_CACHE_NAME;
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.WORK_CACHE_NAME;
/**
@ -51,12 +69,14 @@ public class FeatureEnabledTest extends KeycloakModelTest {
assertEmbeddedCacheDoesNotExists(clusterProvider, WORK_CACHE_NAME);
assertEmbeddedCacheDoesNotExists(clusterProvider, AUTHENTICATION_SESSIONS_CACHE_NAME);
assertEmbeddedCacheDoesNotExists(clusterProvider, ACTION_TOKEN_CACHE);
assertEmbeddedCacheDoesNotExists(clusterProvider, LOGIN_FAILURE_CACHE_NAME);
// TODO [pruivo] all caches eventually won't exists in embedded
Arrays.stream(CLUSTERED_CACHE_NAMES)
.filter(Predicate.not(Predicate.isEqual(WORK_CACHE_NAME)))
.filter(Predicate.not(Predicate.isEqual(AUTHENTICATION_SESSIONS_CACHE_NAME)))
.filter(Predicate.not(Predicate.isEqual(ACTION_TOKEN_CACHE)))
.filter(Predicate.not(Predicate.isEqual(LOGIN_FAILURE_CACHE_NAME)))
.forEach(s -> assertEmbeddedCacheExists(clusterProvider, s));
Arrays.stream(CLUSTERED_CACHE_NAMES).forEach(s -> assertRemoteCacheExists(clusterProvider, s));

View file

@ -0,0 +1,253 @@
/*
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed 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.keycloak.testsuite.model.loginfailure;
import java.util.List;
import java.util.stream.IntStream;
import org.infinispan.client.hotrod.RemoteCache;
import org.junit.Assert;
import org.junit.Assume;
import org.junit.Test;
import org.keycloak.connections.infinispan.InfinispanConnectionProvider;
import org.keycloak.infinispan.util.InfinispanUtils;
import org.keycloak.models.Constants;
import org.keycloak.models.KeycloakSession;
import org.keycloak.models.RealmModel;
import org.keycloak.models.RealmProvider;
import org.keycloak.models.UserLoginFailureProvider;
import org.keycloak.models.UserProvider;
import org.keycloak.models.sessions.infinispan.entities.LoginFailureEntity;
import org.keycloak.models.sessions.infinispan.entities.LoginFailureKey;
import org.keycloak.testsuite.model.KeycloakModelTest;
import org.keycloak.testsuite.model.RequireProvider;
@RequireProvider(UserLoginFailureProvider.class)
@RequireProvider(UserProvider.class)
@RequireProvider(RealmProvider.class)
public class RemoteLoginFailureTest extends KeycloakModelTest {
private static final int NUM_USERS = 10;
private String realmId;
private List<String> userIds;
@Override
public void createEnvironment(KeycloakSession session) {
RealmModel realm = createRealm(session, "remote-login-failure-test");
realm.setDefaultRole(session.roles().addRealmRole(realm, Constants.DEFAULT_ROLES_ROLE_PREFIX + "-" + realm.getName()));
realmId = realm.getId();
userIds = IntStream.range(0, NUM_USERS)
.mapToObj(index -> "user-" + index)
.map(username -> {
var user = session.users().addUser(realm, username);
user.setEmail(username + "@localhost");
return user.getId();
})
.toList();
}
@Override
public void cleanEnvironment(KeycloakSession s) {
s.realms().removeRealm(realmId);
}
@Test
public void testLoginFailureCreation() {
Assume.assumeTrue(InfinispanUtils.isRemoteInfinispan());
var cache = getLoginFailureCache();
inComittedTransaction(session -> {
var realm = session.realms().getRealm(realmId);
var loginFailures = session.loginFailures().addUserLoginFailure(realm, userIds.get(0));
loginFailures.incrementFailures();
});
var entity = cache.get(new LoginFailureKey(realmId, userIds.get(0)));
assertEntity(entity, realmId, userIds.get(0), 0, null, 1, 0, 0);
}
@Test
public void testLoginFailureChangeLog() {
Assume.assumeTrue(InfinispanUtils.isRemoteInfinispan());
var cache = getLoginFailureCache();
var key = new LoginFailureKey(realmId, userIds.get(0));
var entity = new LoginFailureEntity(realmId, userIds.get(0));
entity.setLastFailure(1000);
entity.setNumFailures(2);
entity.setNumTemporaryLockouts(10);
entity.setLastIPFailure("127.0.0.1");
entity.setFailedLoginNotBefore(2000);
cache.put(key, entity);
inComittedTransaction(session -> {
var realm = session.realms().getRealm(realmId);
var loginFailures = session.loginFailures().getUserLoginFailure(realm, userIds.get(0));
// update all fields
loginFailures.setLastFailure(10000);
loginFailures.incrementFailures();
loginFailures.incrementTemporaryLockouts();
loginFailures.setLastIPFailure("127.0.1.1");
loginFailures.setFailedLoginNotBefore(20000);
});
entity = cache.get(key);
assertEntity(entity, realmId, userIds.get(0), 10000, "127.0.1.1", 3, 20000, 11);
}
@Test
public void testLoginFailureChangeLogWithConcurrent() {
Assume.assumeTrue(InfinispanUtils.isRemoteInfinispan());
var cache = getLoginFailureCache();
var key = new LoginFailureKey(realmId, userIds.get(0));
var entity = new LoginFailureEntity(realmId, userIds.get(0));
entity.setLastFailure(1000);
entity.setNumFailures(2);
entity.setNumTemporaryLockouts(10);
entity.setLastIPFailure("127.0.0.1");
entity.setFailedLoginNotBefore(2000);
cache.put(key, entity);
inComittedTransaction(session -> {
var realm = session.realms().getRealm(realmId);
var loginFailures = session.loginFailures().getUserLoginFailure(realm, userIds.get(0));
// update all fields
loginFailures.setLastFailure(10000);
loginFailures.incrementFailures();
loginFailures.incrementTemporaryLockouts();
loginFailures.setLastIPFailure("127.0.1.1");
loginFailures.setFailedLoginNotBefore(20000);
createRandomEntityInCache(cache, 20, 30, realmId, userIds.get(0));
});
entity = cache.get(key);
assertEntity(entity, realmId, userIds.get(0), 10000, "127.0.1.1", 21, 20000, 31);
}
@Test
public void testLoginFailureClear() {
Assume.assumeTrue(InfinispanUtils.isRemoteInfinispan());
var cache = getLoginFailureCache();
var key = new LoginFailureKey(realmId, userIds.get(0));
var entity = new LoginFailureEntity(realmId, userIds.get(0));
entity.setLastFailure(1000);
entity.setNumFailures(2);
entity.setNumTemporaryLockouts(10);
entity.setLastIPFailure("127.0.0.1");
entity.setFailedLoginNotBefore(2000);
cache.put(key, entity);
inComittedTransaction(session -> {
var realm = session.realms().getRealm(realmId);
var loginFailures = session.loginFailures().getUserLoginFailure(realm, userIds.get(0));
loginFailures.incrementTemporaryLockouts();
loginFailures.clearFailures();
// create a conflict? should not make a difference
createRandomEntityInCache(cache, 1, 0, realmId, userIds.get(0));
});
entity = cache.get(key);
assertEntity(entity, realmId, userIds.get(0), 0, null, 0, 0, 0);
}
@Test
public void testLoginFailureRemove() {
Assume.assumeTrue(InfinispanUtils.isRemoteInfinispan());
var cache = getLoginFailureCache();
var key = new LoginFailureKey(realmId, userIds.get(0));
var entity = new LoginFailureEntity(realmId, userIds.get(0));
cache.put(key, entity);
inComittedTransaction(session -> {
var realm = session.realms().getRealm(realmId);
session.loginFailures().removeUserLoginFailure(realm, userIds.get(0));
});
entity = cache.get(key);
Assert.assertNull(entity);
}
@Test
public void testLoginFailureRemoveAll() {
Assume.assumeTrue(InfinispanUtils.isRemoteInfinispan());
var cache = getLoginFailureCache();
// clear garbage from previous tests
cache.clear();
for (var userId : userIds) {
var entity = new LoginFailureEntity(realmId, userId);
cache.put(new LoginFailureKey(realmId, userId), entity);
}
Assert.assertEquals(10, cache.size());
inComittedTransaction(session -> {
var realm = session.realms().getRealm(realmId);
session.loginFailures().removeAllUserLoginFailures(realm);
});
Assert.assertEquals(0, cache.size());
}
private static void createRandomEntityInCache(RemoteCache<LoginFailureKey, LoginFailureEntity> cache, int failures, int temporaryLockouts, String realmId, String userId) {
var key = new LoginFailureKey(realmId, userId);
var entity = new LoginFailureEntity(realmId, userId);
entity.setLastFailure(5000); // does not matter
entity.setNumFailures(failures);
entity.setNumTemporaryLockouts(temporaryLockouts);
entity.setLastIPFailure("127.0.0.1");
entity.setFailedLoginNotBefore(5000);
cache.put(key, entity);
}
private static void assertEntity(LoginFailureEntity entity, String realmId, String userId, long lastFailure, String lastIpFailure, int failures, int failedLoginNotBefore, int temporaryLockouts) {
Assert.assertNotNull(entity);
Assert.assertEquals(realmId, entity.getRealmId());
Assert.assertEquals(userId, entity.getUserId());
Assert.assertEquals(lastFailure, entity.getLastFailure());
Assert.assertEquals(lastIpFailure, entity.getLastIPFailure());
Assert.assertEquals(failures, entity.getNumFailures());
Assert.assertEquals(failedLoginNotBefore, entity.getFailedLoginNotBefore());
Assert.assertEquals(temporaryLockouts, entity.getNumTemporaryLockouts());
}
private RemoteCache<LoginFailureKey, LoginFailureEntity> getLoginFailureCache() {
return getInfinispanConnectionProvider().getRemoteCache(InfinispanConnectionProvider.LOGIN_FAILURE_CACHE_NAME);
}
private InfinispanConnectionProvider getInfinispanConnectionProvider() {
return inComittedTransaction(RemoteLoginFailureTest::getInfinispanConnectionProviderWithSession);
}
private static InfinispanConnectionProvider getInfinispanConnectionProviderWithSession(KeycloakSession session) {
return session.getProvider(InfinispanConnectionProvider.class);
}
}

View file

@ -1,5 +1,5 @@
/*
* Copyright 2021 Red Hat, Inc. and/or its affiliates
* Copyright 2024 Red Hat, Inc. and/or its affiliates
* and other contributors as indicated by the @author tags.
*
* Licensed under the Apache License, Version 2.0 (the "License");
@ -16,6 +16,10 @@
*/
package org.keycloak.testsuite.model.parameters;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Stream;
import com.google.common.collect.ImmutableSet;
import org.junit.runner.Description;
import org.junit.runners.model.Statement;
@ -26,15 +30,12 @@ import org.keycloak.models.UserSessionSpi;
import org.keycloak.models.sessions.infinispan.remote.RemoteInfinispanAuthenticationSessionProviderFactory;
import org.keycloak.models.sessions.infinispan.remote.RemoteInfinispanSingleUseObjectProviderFactory;
import org.keycloak.models.sessions.infinispan.remote.RemoteStickySessionEncoderProviderFactory;
import org.keycloak.models.sessions.infinispan.remote.RemoteUserLoginFailureProviderFactory;
import org.keycloak.provider.ProviderFactory;
import org.keycloak.testsuite.model.Config;
import org.keycloak.testsuite.model.HotRodServerRule;
import org.keycloak.testsuite.model.KeycloakModelParameters;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Stream;
/**
* Copied from {@link CrossDCInfinispan}.
* <p>
@ -59,6 +60,7 @@ public class RemoteInfinispan extends KeycloakModelParameters {
.add(RemoteInfinispanSingleUseObjectProviderFactory.class)
.add(RemoteStickySessionEncoderProviderFactory.class)
.add(RemoteLoadBalancerCheckProviderFactory.class)
.add(RemoteUserLoginFailureProviderFactory.class)
.build();
@Override

View file

@ -84,29 +84,9 @@ public class SessionTimeoutsTest extends KeycloakModelTest {
s.sessions().getOfflineUserSessionsStream(realm, user1).forEach(us -> s.sessions().removeOfflineUserSession(realm, us));
s.realms().removeRealm(realmId);
// explicitly clear session caches, as removeUserSessions() contains asynchronous processing or might be incomplete due to a previous failure
clearSessionCaches(s);
super.cleanEnvironment(s);
}
private void clearSessionCaches(KeycloakSession s) {
InfinispanConnectionProvider provider = s.getProvider(InfinispanConnectionProvider.class);
if (provider != null) {
for (String cache : InfinispanConnectionProvider.CLUSTERED_CACHE_NAMES) {
provider.getCache(cache).clear();
}
}
HotRodServerRule hotRodServer = getParameters(HotRodServerRule.class).findFirst().orElse(null);
if (hotRodServer != null) {
for (String cache : InfinispanConnectionProvider.CLUSTERED_CACHE_NAMES) {
hotRodServer.getHotRodCacheManager().getCache(cache).clear();
hotRodServer.getHotRodCacheManager2().getCache(cache).clear();
}
}
}
protected static UserSessionModel createUserSession(KeycloakSession session, RealmModel realm, UserModel user, boolean offline) {
UserSessionModel userSession = session.sessions().createUserSession(UUID.randomUUID().toString(), realm, user, "user1", "127.0.0.1",
"form", true, null, null, UserSessionModel.SessionPersistenceState.PERSISTENT);
@ -345,6 +325,7 @@ public class SessionTimeoutsTest extends KeycloakModelTest {
Assert.assertNull(getUserSession(session, realm, sessions[0], offline));
return null;
});
processExpiration(offline);
} finally {
setTimeOffset(0);
}
@ -370,12 +351,12 @@ public class SessionTimeoutsTest extends KeycloakModelTest {
testUserClientMaxLifespanSmallerThanSession(true, true);
}
@Test(timeout = 10 * 1000)
@Test
public void testOfflineUserClientIdleTimeoutSmallerThanSessionNoRefresh() {
testUserClientIdleTimeoutSmallerThanSession(0, true, false);
}
@Test(timeout = 10 * 1000)
@Test
public void testOfflineUserClientIdleTimeoutSmallerThanSessionOneRefresh() {
testUserClientIdleTimeoutSmallerThanSession(1, true, false);
}
@ -400,12 +381,12 @@ public class SessionTimeoutsTest extends KeycloakModelTest {
testUserClientMaxLifespanSmallerThanSession(false, true);
}
@Test(timeout = 10 * 1000)
@Test
public void testOnlineUserClientIdleTimeoutSmallerThanSessionNoRefresh() {
testUserClientIdleTimeoutSmallerThanSession(0, false, false);
}
@Test(timeout = 10 * 1000)
@Test
public void testOnlineUserClientIdleTimeoutSmallerThanSessionOneRefresh() {
testUserClientIdleTimeoutSmallerThanSession(1, false, false);
}
@ -416,12 +397,26 @@ public class SessionTimeoutsTest extends KeycloakModelTest {
* @param offline boolean Indicates where we work with offline sessions
*/
private void allowXSiteReplication(boolean offline) {
HotRodServerRule hotRodServer = getParameters(HotRodServerRule.class).findFirst().orElse(null);
if (hotRodServer != null) {
var cacheName = offline ? InfinispanConnectionProvider.OFFLINE_CLIENT_SESSION_CACHE_NAME : InfinispanConnectionProvider.CLIENT_SESSION_CACHE_NAME;
var cache1 = hotRodServer.getHotRodCacheManager().getCache(cacheName);
var cache2 = hotRodServer.getHotRodCacheManager2().getCache(cacheName);
eventually(null, () -> cache1.size() == cache2.size(), 10000, 10, TimeUnit.MILLISECONDS);
var hotRodServer = getParameters(HotRodServerRule.class).findFirst();
if (hotRodServer.isEmpty()) {
return;
}
var cacheName = offline ? InfinispanConnectionProvider.OFFLINE_CLIENT_SESSION_CACHE_NAME : InfinispanConnectionProvider.CLIENT_SESSION_CACHE_NAME;
var cache1 = hotRodServer.get().getHotRodCacheManager().getCache(cacheName);
var cache2 = hotRodServer.get().getHotRodCacheManager2().getCache(cacheName);
eventually(() -> "Wrong cache size. Site1: " + cache1.keySet() + ", Site2: " + cache2.keySet(),
() -> cache1.size() == cache2.size(), 10000, 10, TimeUnit.MILLISECONDS);
}
private void processExpiration(boolean offline) {
var hotRodServer = getParameters(HotRodServerRule.class).findFirst();
if (hotRodServer.isEmpty()) {
return;
}
// force expired entries to be removed from memory
var cacheName = offline ? InfinispanConnectionProvider.OFFLINE_CLIENT_SESSION_CACHE_NAME : InfinispanConnectionProvider.CLIENT_SESSION_CACHE_NAME;
hotRodServer.get().getHotRodCacheManager().getCache(cacheName).getAdvancedCache().getExpirationManager().processExpiration();
hotRodServer.get().getHotRodCacheManager2().getCache(cacheName).getAdvancedCache().getExpirationManager().processExpiration();
}
}