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:
parent
833aad661e
commit
9006218559
48 changed files with 1787 additions and 217 deletions
41
.github/workflows/ci.yml
vendored
41
.github/workflows/ci.yml
vendored
|
@ -358,6 +358,46 @@ jobs:
|
||||||
name: store-it-mvn-logs
|
name: store-it-mvn-logs
|
||||||
path: .github/scripts/ansible/files
|
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:
|
store-integration-tests:
|
||||||
name: Store IT
|
name: Store IT
|
||||||
needs: [build, conditional]
|
needs: [build, conditional]
|
||||||
|
@ -823,6 +863,7 @@ jobs:
|
||||||
- webauthn-integration-tests
|
- webauthn-integration-tests
|
||||||
- sssd-unit-tests
|
- sssd-unit-tests
|
||||||
- migration-tests
|
- migration-tests
|
||||||
|
- external-infinispan-tests
|
||||||
runs-on: ubuntu-latest
|
runs-on: ubuntu-latest
|
||||||
steps:
|
steps:
|
||||||
- uses: actions/checkout@v4
|
- uses: actions/checkout@v4
|
||||||
|
|
|
@ -17,6 +17,14 @@
|
||||||
|
|
||||||
package org.keycloak.cluster.infinispan;
|
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.Cache;
|
||||||
import org.infinispan.client.hotrod.exceptions.HotRodClientException;
|
import org.infinispan.client.hotrod.exceptions.HotRodClientException;
|
||||||
import org.infinispan.lifecycle.ComponentStatus;
|
import org.infinispan.lifecycle.ComponentStatus;
|
||||||
|
@ -38,21 +46,14 @@ import org.keycloak.connections.infinispan.TopologyInfo;
|
||||||
import org.keycloak.infinispan.util.InfinispanUtils;
|
import org.keycloak.infinispan.util.InfinispanUtils;
|
||||||
import org.keycloak.models.KeycloakSession;
|
import org.keycloak.models.KeycloakSession;
|
||||||
import org.keycloak.models.KeycloakSessionFactory;
|
import org.keycloak.models.KeycloakSessionFactory;
|
||||||
|
import org.keycloak.provider.EnvironmentDependentProviderFactory;
|
||||||
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;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This impl is aware of Cross-Data-Center scenario too
|
* This impl is aware of Cross-Data-Center scenario too
|
||||||
*
|
*
|
||||||
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
|
* @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);
|
protected static final Logger logger = Logger.getLogger(InfinispanClusterProviderFactory.class);
|
||||||
|
|
||||||
|
|
|
@ -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;
|
package org.keycloak.cluster.infinispan.remote;
|
||||||
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;
|
|
||||||
|
|
||||||
import java.lang.invoke.MethodHandles;
|
import java.lang.invoke.MethodHandles;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
@ -22,27 +28,31 @@ import java.util.concurrent.Future;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.function.Supplier;
|
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.InfinispanClusterProvider.TASK_KEY_PREFIX;
|
||||||
import static org.keycloak.cluster.infinispan.remote.RemoteInfinispanClusterProviderFactory.putIfAbsentWithRetries;
|
import static org.keycloak.cluster.infinispan.remote.RemoteInfinispanClusterProviderFactory.putIfAbsentWithRetries;
|
||||||
|
|
||||||
public class RemoteInfinispanClusterProvider implements ClusterProvider {
|
public class RemoteInfinispanClusterProvider implements ClusterProvider {
|
||||||
|
|
||||||
private static final Logger logger = Logger.getLogger(MethodHandles.lookup().lookupClass());
|
private static final Logger logger = Logger.getLogger(MethodHandles.lookup().lookupClass());
|
||||||
private final int clusterStartupTime;
|
private final SharedData data;
|
||||||
private final RemoteCache<String, LockEntry> cache;
|
|
||||||
private final RemoteInfinispanNotificationManager notificationManager;
|
|
||||||
private final Executor executor;
|
|
||||||
|
|
||||||
public RemoteInfinispanClusterProvider(int clusterStartupTime, RemoteCache<String, LockEntry> cache, RemoteInfinispanNotificationManager notificationManager, Executor executor) {
|
public RemoteInfinispanClusterProvider(SharedData data) {
|
||||||
this.clusterStartupTime = clusterStartupTime;
|
this.data = Objects.requireNonNull(data);
|
||||||
this.cache = Objects.requireNonNull(cache);
|
|
||||||
this.notificationManager = Objects.requireNonNull(notificationManager);
|
|
||||||
this.executor = Objects.requireNonNull(executor);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int getClusterStartupTime() {
|
public int getClusterStartupTime() {
|
||||||
return clusterStartupTime;
|
return data.clusterStartupTime();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -70,7 +80,7 @@ public class RemoteInfinispanClusterProvider implements ClusterProvider {
|
||||||
@Override
|
@Override
|
||||||
public Future<Boolean> executeIfNotExecutedAsync(String taskKey, int taskTimeoutInSeconds, Callable task) {
|
public Future<Boolean> executeIfNotExecutedAsync(String taskKey, int taskTimeoutInSeconds, Callable task) {
|
||||||
TaskCallback newCallback = new TaskCallback();
|
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
|
// We successfully submitted our task
|
||||||
if (newCallback == callback) {
|
if (newCallback == callback) {
|
||||||
|
@ -89,7 +99,7 @@ public class RemoteInfinispanClusterProvider implements ClusterProvider {
|
||||||
return callback.isSuccess();
|
return callback.isSuccess();
|
||||||
};
|
};
|
||||||
|
|
||||||
callback.setFuture(CompletableFuture.supplyAsync(wrappedTask, executor));
|
callback.setFuture(CompletableFuture.supplyAsync(wrappedTask, data.executor()));
|
||||||
} else {
|
} else {
|
||||||
logger.infof("Task already in progress on this cluster node. Will wait until it's finished");
|
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
|
@Override
|
||||||
public void registerListener(String taskKey, ClusterListener task) {
|
public void registerListener(String taskKey, ClusterListener task) {
|
||||||
notificationManager.registerListener(taskKey, task);
|
data.notificationManager().registerListener(taskKey, task);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void notify(String taskKey, ClusterEvent event, boolean ignoreSender, DCNotify dcNotify) {
|
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
|
@Override
|
||||||
public void notify(String taskKey, Collection<? extends ClusterEvent> events, boolean ignoreSender, DCNotify dcNotify) {
|
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
|
@Override
|
||||||
|
@ -120,7 +130,7 @@ public class RemoteInfinispanClusterProvider implements ClusterProvider {
|
||||||
private boolean tryLock(String cacheKey, int taskTimeoutInSeconds) {
|
private boolean tryLock(String cacheKey, int taskTimeoutInSeconds) {
|
||||||
LockEntry myLock = createLockEntry();
|
LockEntry myLock = createLockEntry();
|
||||||
|
|
||||||
LockEntry existingLock = putIfAbsentWithRetries(cache, cacheKey, myLock, taskTimeoutInSeconds);
|
LockEntry existingLock = putIfAbsentWithRetries(data.cache(), cacheKey, myLock, taskTimeoutInSeconds);
|
||||||
if (existingLock != null) {
|
if (existingLock != null) {
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.tracef("Task %s in progress already by node %s. Ignoring task.", cacheKey, existingLock.node());
|
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() {
|
private LockEntry createLockEntry() {
|
||||||
return new LockEntry(notificationManager.getMyNodeName());
|
return new LockEntry(data.notificationManager().getMyNodeName());
|
||||||
}
|
}
|
||||||
|
|
||||||
private void removeFromCache(String cacheKey) {
|
private void removeFromCache(String cacheKey) {
|
||||||
// More attempts to send the message (it may fail if some node fails in the meantime)
|
// More attempts to send the message (it may fail if some node fails in the meantime)
|
||||||
Retry.executeWithBackoff((int iteration) -> {
|
Retry.executeWithBackoff((int iteration) -> {
|
||||||
cache.remove(cacheKey);
|
data.cache().remove(cacheKey);
|
||||||
if (logger.isTraceEnabled()) {
|
if (logger.isTraceEnabled()) {
|
||||||
logger.tracef("Task %s removed from the cache", cacheKey);
|
logger.tracef("Task %s removed from the cache", cacheKey);
|
||||||
}
|
}
|
||||||
}, 10, 10);
|
}, 10, 10);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public interface SharedData {
|
||||||
|
int clusterStartupTime();
|
||||||
|
RemoteCache<String, LockEntry> cache();
|
||||||
|
RemoteInfinispanNotificationManager notificationManager();
|
||||||
|
Executor executor();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
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.RemoteCache;
|
||||||
import org.infinispan.client.hotrod.exceptions.HotRodClientException;
|
import org.infinispan.client.hotrod.exceptions.HotRodClientException;
|
||||||
import org.infinispan.commons.util.ByRef;
|
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.Retry;
|
||||||
import org.keycloak.common.util.Time;
|
import org.keycloak.common.util.Time;
|
||||||
import org.keycloak.connections.infinispan.InfinispanConnectionProvider;
|
import org.keycloak.connections.infinispan.InfinispanConnectionProvider;
|
||||||
import org.keycloak.connections.infinispan.TopologyInfo;
|
|
||||||
import org.keycloak.infinispan.util.InfinispanUtils;
|
import org.keycloak.infinispan.util.InfinispanUtils;
|
||||||
import org.keycloak.models.KeycloakSession;
|
import org.keycloak.models.KeycloakSession;
|
||||||
import org.keycloak.models.KeycloakSessionFactory;
|
import org.keycloak.models.KeycloakSessionFactory;
|
||||||
|
import org.keycloak.provider.EnvironmentDependentProviderFactory;
|
||||||
import java.io.Serializable;
|
|
||||||
import java.lang.invoke.MethodHandles;
|
|
||||||
import java.util.concurrent.Executor;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
|
||||||
|
|
||||||
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.WORK_CACHE_NAME;
|
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());
|
private static final Logger logger = Logger.getLogger(MethodHandles.lookup().lookupClass());
|
||||||
|
|
||||||
|
@ -35,10 +51,14 @@ public class RemoteInfinispanClusterProviderFactory implements ClusterProviderFa
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ClusterProvider create(KeycloakSession session) {
|
public ClusterProvider create(KeycloakSession session) {
|
||||||
|
if (workCache == null) {
|
||||||
|
// Keycloak does not ensure postInit() is invoked before create()
|
||||||
|
lazyInit(session);
|
||||||
|
}
|
||||||
assert workCache != null;
|
assert workCache != null;
|
||||||
assert notificationManager != null;
|
assert notificationManager != null;
|
||||||
assert executor != null;
|
assert executor != null;
|
||||||
return new RemoteInfinispanClusterProvider(clusterStartupTime, workCache, notificationManager, executor);
|
return new RemoteInfinispanClusterProvider(this);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -47,16 +67,9 @@ public class RemoteInfinispanClusterProviderFactory implements ClusterProviderFa
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized void postInit(KeycloakSessionFactory factory) {
|
public void postInit(KeycloakSessionFactory factory) {
|
||||||
try (var session = factory.create()) {
|
try (var session = factory.create()) {
|
||||||
var ispnProvider = session.getProvider(InfinispanConnectionProvider.class);
|
lazyInit(session);
|
||||||
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));
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -82,10 +95,18 @@ public class RemoteInfinispanClusterProviderFactory implements ClusterProviderFa
|
||||||
return InfinispanUtils.isRemoteInfinispan();
|
return InfinispanUtils.isRemoteInfinispan();
|
||||||
}
|
}
|
||||||
|
|
||||||
private static TopologyInfo getTopologyInfo(KeycloakSessionFactory factory) {
|
private synchronized void lazyInit(KeycloakSession session) {
|
||||||
try (var session = factory.create()) {
|
if (workCache != null) {
|
||||||
return session.getProvider(InfinispanConnectionProvider.class).getTopologyInfo();
|
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) {
|
private static int initClusterStartupTime(RemoteCache<String, Integer> cache, int serverStartupTime) {
|
||||||
|
@ -93,7 +114,6 @@ public class RemoteInfinispanClusterProviderFactory implements ClusterProviderFa
|
||||||
return clusterStartupTime == null ? serverStartupTime : clusterStartupTime;
|
return clusterStartupTime == null ? serverStartupTime : clusterStartupTime;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
static <V> V putIfAbsentWithRetries(RemoteCache<String, V> workCache, String key, V value, int taskTimeoutInSeconds) {
|
static <V> V putIfAbsentWithRetries(RemoteCache<String, V> workCache, String key, V value, int taskTimeoutInSeconds) {
|
||||||
ByRef<V> ref = new ByRef<>(null);
|
ByRef<V> ref = new ByRef<>(null);
|
||||||
|
|
||||||
|
@ -115,4 +135,24 @@ public class RemoteInfinispanClusterProviderFactory implements ClusterProviderFa
|
||||||
|
|
||||||
return ref.get();
|
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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
package org.keycloak.cluster.infinispan.remote;
|
||||||
|
|
||||||
import java.lang.invoke.MethodHandles;
|
import java.lang.invoke.MethodHandles;
|
||||||
|
|
|
@ -339,9 +339,9 @@ public class DefaultInfinispanConnectionProviderFactory implements InfinispanCon
|
||||||
defineClusteredCache(cacheManager, OFFLINE_USER_SESSION_CACHE_NAME, clusteredConfiguration);
|
defineClusteredCache(cacheManager, OFFLINE_USER_SESSION_CACHE_NAME, clusteredConfiguration);
|
||||||
defineClusteredCache(cacheManager, CLIENT_SESSION_CACHE_NAME, clusteredConfiguration);
|
defineClusteredCache(cacheManager, CLIENT_SESSION_CACHE_NAME, clusteredConfiguration);
|
||||||
defineClusteredCache(cacheManager, OFFLINE_CLIENT_SESSION_CACHE_NAME, clusteredConfiguration);
|
defineClusteredCache(cacheManager, OFFLINE_CLIENT_SESSION_CACHE_NAME, clusteredConfiguration);
|
||||||
defineClusteredCache(cacheManager, LOGIN_FAILURE_CACHE_NAME, clusteredConfiguration);
|
|
||||||
|
|
||||||
if (InfinispanUtils.isEmbeddedInfinispan()) {
|
if (InfinispanUtils.isEmbeddedInfinispan()) {
|
||||||
|
defineClusteredCache(cacheManager, LOGIN_FAILURE_CACHE_NAME, clusteredConfiguration);
|
||||||
defineClusteredCache(cacheManager, AUTHENTICATION_SESSIONS_CACHE_NAME, clusteredConfiguration);
|
defineClusteredCache(cacheManager, AUTHENTICATION_SESSIONS_CACHE_NAME, clusteredConfiguration);
|
||||||
|
|
||||||
var actionTokenBuilder = getActionTokenCacheConfig();
|
var actionTokenBuilder = getActionTokenCacheConfig();
|
||||||
|
|
|
@ -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;
|
package org.keycloak.connections.infinispan.remote;
|
||||||
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
|
|
@ -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;
|
package org.keycloak.connections.infinispan.remote;
|
||||||
|
|
||||||
import org.infinispan.client.hotrod.impl.InternalRemoteCache;
|
import org.infinispan.client.hotrod.impl.InternalRemoteCache;
|
||||||
|
|
|
@ -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;
|
package org.keycloak.infinispan.util;
|
||||||
|
|
||||||
import org.keycloak.common.Profile;
|
import org.keycloak.common.Profile;
|
||||||
|
|
|
@ -17,6 +17,11 @@
|
||||||
|
|
||||||
package org.keycloak.models.sessions.infinispan;
|
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.infinispan.Cache;
|
||||||
import org.jboss.logging.Logger;
|
import org.jboss.logging.Logger;
|
||||||
import org.keycloak.Config;
|
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.sessions.infinispan.util.InfinispanKeyGenerator;
|
||||||
import org.keycloak.models.utils.KeycloakModelUtils;
|
import org.keycloak.models.utils.KeycloakModelUtils;
|
||||||
import org.keycloak.models.utils.PostMigrationEvent;
|
import org.keycloak.models.utils.PostMigrationEvent;
|
||||||
|
import org.keycloak.provider.EnvironmentDependentProviderFactory;
|
||||||
import org.keycloak.provider.ProviderConfigProperty;
|
import org.keycloak.provider.ProviderConfigProperty;
|
||||||
import org.keycloak.provider.ProviderConfigurationBuilder;
|
import org.keycloak.provider.ProviderConfigurationBuilder;
|
||||||
import org.keycloak.provider.ProviderEvent;
|
import org.keycloak.provider.ProviderEvent;
|
||||||
import org.keycloak.provider.ProviderEventListener;
|
import org.keycloak.provider.ProviderEventListener;
|
||||||
import org.keycloak.sessions.AuthenticationSessionProviderFactory;
|
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>
|
* @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);
|
private static final Logger log = Logger.getLogger(InfinispanAuthenticationSessionProviderFactory.class);
|
||||||
|
|
||||||
|
|
|
@ -18,6 +18,8 @@
|
||||||
package org.keycloak.models.sessions.infinispan;
|
package org.keycloak.models.sessions.infinispan;
|
||||||
|
|
||||||
|
|
||||||
|
import java.util.function.Supplier;
|
||||||
|
|
||||||
import org.infinispan.Cache;
|
import org.infinispan.Cache;
|
||||||
import org.infinispan.client.hotrod.Flag;
|
import org.infinispan.client.hotrod.Flag;
|
||||||
import org.infinispan.client.hotrod.RemoteCache;
|
import org.infinispan.client.hotrod.RemoteCache;
|
||||||
|
@ -31,13 +33,12 @@ import org.keycloak.models.KeycloakSession;
|
||||||
import org.keycloak.models.KeycloakSessionFactory;
|
import org.keycloak.models.KeycloakSessionFactory;
|
||||||
import org.keycloak.models.SingleUseObjectProviderFactory;
|
import org.keycloak.models.SingleUseObjectProviderFactory;
|
||||||
import org.keycloak.models.sessions.infinispan.entities.SingleUseObjectValueEntity;
|
import org.keycloak.models.sessions.infinispan.entities.SingleUseObjectValueEntity;
|
||||||
|
import org.keycloak.provider.EnvironmentDependentProviderFactory;
|
||||||
import java.util.function.Supplier;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
|
* @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);
|
private static final Logger LOG = Logger.getLogger(InfinispanSingleUseObjectProviderFactory.class);
|
||||||
|
|
||||||
|
|
|
@ -17,22 +17,23 @@
|
||||||
|
|
||||||
package org.keycloak.models.sessions.infinispan;
|
package org.keycloak.models.sessions.infinispan;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
import org.jboss.logging.Logger;
|
import org.jboss.logging.Logger;
|
||||||
import org.keycloak.Config;
|
import org.keycloak.Config;
|
||||||
import org.keycloak.infinispan.util.InfinispanUtils;
|
import org.keycloak.infinispan.util.InfinispanUtils;
|
||||||
import org.keycloak.models.KeycloakSession;
|
import org.keycloak.models.KeycloakSession;
|
||||||
import org.keycloak.models.KeycloakSessionFactory;
|
import org.keycloak.models.KeycloakSessionFactory;
|
||||||
|
import org.keycloak.provider.EnvironmentDependentProviderFactory;
|
||||||
import org.keycloak.provider.ProviderConfigProperty;
|
import org.keycloak.provider.ProviderConfigProperty;
|
||||||
import org.keycloak.provider.ProviderConfigurationBuilder;
|
import org.keycloak.provider.ProviderConfigurationBuilder;
|
||||||
import org.keycloak.sessions.StickySessionEncoderProvider;
|
import org.keycloak.sessions.StickySessionEncoderProvider;
|
||||||
import org.keycloak.sessions.StickySessionEncoderProviderFactory;
|
import org.keycloak.sessions.StickySessionEncoderProviderFactory;
|
||||||
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
|
* @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);
|
private static final Logger log = Logger.getLogger(InfinispanStickySessionEncoderProviderFactory.class);
|
||||||
|
|
||||||
|
|
|
@ -49,13 +49,14 @@ import org.keycloak.models.sessions.infinispan.remotestore.RemoteCacheSessionsLo
|
||||||
import org.keycloak.models.sessions.infinispan.util.SessionTimeouts;
|
import org.keycloak.models.sessions.infinispan.util.SessionTimeouts;
|
||||||
import org.keycloak.models.utils.KeycloakModelUtils;
|
import org.keycloak.models.utils.KeycloakModelUtils;
|
||||||
import org.keycloak.models.utils.PostMigrationEvent;
|
import org.keycloak.models.utils.PostMigrationEvent;
|
||||||
|
import org.keycloak.provider.EnvironmentDependentProviderFactory;
|
||||||
|
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @author <a href="mailto:mkanis@redhat.com">Martin Kanis</a>
|
* @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);
|
private static final Logger log = Logger.getLogger(InfinispanUserLoginFailureProviderFactory.class);
|
||||||
|
|
||||||
|
@ -69,7 +70,7 @@ public class InfinispanUserLoginFailureProviderFactory implements UserLoginFailu
|
||||||
SerializeExecutionsByKey<LoginFailureKey> serializer = new SerializeExecutionsByKey<>();
|
SerializeExecutionsByKey<LoginFailureKey> serializer = new SerializeExecutionsByKey<>();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public UserLoginFailureProvider create(KeycloakSession session) {
|
public InfinispanUserLoginFailureProvider create(KeycloakSession session) {
|
||||||
InfinispanConnectionProvider connections = session.getProvider(InfinispanConnectionProvider.class);
|
InfinispanConnectionProvider connections = session.getProvider(InfinispanConnectionProvider.class);
|
||||||
Cache<LoginFailureKey, SessionEntityWrapper<LoginFailureEntity>> loginFailures = connections.getCache(InfinispanConnectionProvider.LOGIN_FAILURE_CACHE_NAME);
|
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) -> {
|
KeycloakModelUtils.runJobInTransaction(factory, (KeycloakSession session) -> {
|
||||||
checkRemoteCaches(session);
|
checkRemoteCaches(session);
|
||||||
registerClusterListeners(session);
|
registerClusterListeners(session);
|
||||||
// TODO [pruivo] to remove: workaround to run the testsuite.
|
loadLoginFailuresFromRemoteCaches(session);
|
||||||
if (InfinispanUtils.isEmbeddedInfinispan()) {
|
|
||||||
loadLoginFailuresFromRemoteCaches(session);
|
|
||||||
}
|
|
||||||
});
|
});
|
||||||
} else if (event instanceof UserModel.UserRemovedEvent) {
|
} else if (event instanceof UserModel.UserRemovedEvent userRemovedEvent) {
|
||||||
UserModel.UserRemovedEvent userRemovedEvent = (UserModel.UserRemovedEvent) event;
|
|
||||||
|
|
||||||
UserLoginFailureProvider provider = userRemovedEvent.getKeycloakSession().getProvider(UserLoginFailureProvider.class, getId());
|
UserLoginFailureProvider provider = userRemovedEvent.getKeycloakSession().getProvider(UserLoginFailureProvider.class, getId());
|
||||||
provider.removeUserLoginFailure(userRemovedEvent.getRealm(), userRemovedEvent.getUser().getId());
|
provider.removeUserLoginFailure(userRemovedEvent.getRealm(), userRemovedEvent.getUser().getId());
|
||||||
}
|
}
|
||||||
|
@ -223,4 +219,9 @@ public class InfinispanUserLoginFailureProviderFactory implements UserLoginFailu
|
||||||
public int order() {
|
public int order() {
|
||||||
return InfinispanUtils.PROVIDER_ORDER;
|
return InfinispanUtils.PROVIDER_ORDER;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isSupported(Config.Scope config) {
|
||||||
|
return InfinispanUtils.isEmbeddedInfinispan();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
package org.keycloak.models.sessions.infinispan;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -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));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -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;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -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);
|
||||||
|
}
|
|
@ -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);
|
||||||
|
|
||||||
|
}
|
|
@ -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);
|
||||||
|
}
|
|
@ -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;
|
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.cluster.ClusterProvider;
|
||||||
import org.keycloak.common.util.Time;
|
import org.keycloak.common.util.Time;
|
||||||
import org.keycloak.models.ClientModel;
|
import org.keycloak.models.ClientModel;
|
||||||
|
@ -16,10 +38,6 @@ import org.keycloak.sessions.AuthenticationSessionCompoundId;
|
||||||
import org.keycloak.sessions.AuthenticationSessionProvider;
|
import org.keycloak.sessions.AuthenticationSessionProvider;
|
||||||
import org.keycloak.sessions.RootAuthenticationSessionModel;
|
import org.keycloak.sessions.RootAuthenticationSessionModel;
|
||||||
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Objects;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
|
||||||
|
|
||||||
public class RemoteInfinispanAuthenticationSessionProvider implements AuthenticationSessionProvider {
|
public class RemoteInfinispanAuthenticationSessionProvider implements AuthenticationSessionProvider {
|
||||||
|
|
||||||
private final KeycloakSession session;
|
private final KeycloakSession session;
|
||||||
|
@ -78,15 +96,7 @@ public class RemoteInfinispanAuthenticationSessionProvider implements Authentica
|
||||||
@Override
|
@Override
|
||||||
public void onRealmRemoved(RealmModel realm) {
|
public void onRealmRemoved(RealmModel realm) {
|
||||||
// TODO [pruivo] [optimization] with protostream, use delete by query: DELETE FROM ...
|
// TODO [pruivo] [optimization] with protostream, use delete by query: DELETE FROM ...
|
||||||
var cache = transaction.getCache();
|
transaction.removeIf(new RealmFilter(realm.getId()));
|
||||||
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());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -132,4 +142,12 @@ public class RemoteInfinispanAuthenticationSessionProvider implements Authentica
|
||||||
transaction.remove(entity.getId());
|
transaction.remove(entity.getId());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private record RealmFilter(String realmId) implements Predicate<RootAuthenticationSessionEntity> {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean test(RootAuthenticationSessionEntity entity) {
|
||||||
|
return Objects.equals(realmId, entity.getRealmId());
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
package org.keycloak.models.sessions.infinispan.remote;
|
||||||
|
|
||||||
import java.lang.invoke.MethodHandles;
|
import java.lang.invoke.MethodHandles;
|
||||||
|
@ -11,6 +28,7 @@ import org.keycloak.models.KeycloakSession;
|
||||||
import org.keycloak.models.KeycloakSessionFactory;
|
import org.keycloak.models.KeycloakSessionFactory;
|
||||||
import org.keycloak.models.sessions.infinispan.InfinispanAuthenticationSessionProviderFactory;
|
import org.keycloak.models.sessions.infinispan.InfinispanAuthenticationSessionProviderFactory;
|
||||||
import org.keycloak.models.sessions.infinispan.entities.RootAuthenticationSessionEntity;
|
import org.keycloak.models.sessions.infinispan.entities.RootAuthenticationSessionEntity;
|
||||||
|
import org.keycloak.provider.EnvironmentDependentProviderFactory;
|
||||||
import org.keycloak.provider.ProviderConfigProperty;
|
import org.keycloak.provider.ProviderConfigProperty;
|
||||||
import org.keycloak.provider.ProviderConfigurationBuilder;
|
import org.keycloak.provider.ProviderConfigurationBuilder;
|
||||||
import org.keycloak.sessions.AuthenticationSessionProviderFactory;
|
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.connections.infinispan.InfinispanConnectionProvider.getRemoteCache;
|
||||||
import static org.keycloak.models.sessions.infinispan.InfinispanAuthenticationSessionProviderFactory.DEFAULT_AUTH_SESSIONS_LIMIT;
|
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());
|
private final static Logger logger = Logger.getLogger(MethodHandles.lookup().lookupClass());
|
||||||
|
|
||||||
|
|
|
@ -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;
|
package org.keycloak.models.sessions.infinispan.remote;
|
||||||
import org.infinispan.commons.util.concurrent.AggregateCompletionStage;
|
|
||||||
import org.infinispan.commons.util.concurrent.CompletionStages;
|
|
||||||
import org.jboss.logging.Logger;
|
|
||||||
import org.keycloak.models.KeycloakTransaction;
|
|
||||||
|
|
||||||
import java.lang.invoke.MethodHandles;
|
import java.lang.invoke.MethodHandles;
|
||||||
import java.util.LinkedHashMap;
|
import java.util.LinkedHashMap;
|
||||||
|
@ -12,67 +23,53 @@ import java.util.Map;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.CompletionStage;
|
import java.util.concurrent.CompletionStage;
|
||||||
import java.util.concurrent.ExecutionException;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
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 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 Map<K, Operation<K, V>> tasks = new LinkedHashMap<>();
|
||||||
private final RemoteCache<K, V> cache;
|
private final RemoteCache<K, V> cache;
|
||||||
|
private Predicate<V> removePredicate;
|
||||||
|
|
||||||
public RemoteInfinispanKeycloakTransaction(RemoteCache<K, V> cache) {
|
public RemoteInfinispanKeycloakTransaction(RemoteCache<K, V> cache) {
|
||||||
this.cache = Objects.requireNonNull(cache);
|
this.cache = Objects.requireNonNull(cache);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void begin() {
|
protected void commitImpl() {
|
||||||
active = true;
|
|
||||||
tasks.clear();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void commit() {
|
|
||||||
active = false;
|
|
||||||
if (rollback) {
|
|
||||||
throw new RuntimeException("Rollback only!");
|
|
||||||
}
|
|
||||||
AggregateCompletionStage<Void> stage = CompletionStages.aggregateCompletionStage();
|
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()
|
tasks.values().stream()
|
||||||
|
.filter(this::shouldCommitOperation)
|
||||||
.map(this::commitOperation)
|
.map(this::commitOperation)
|
||||||
.forEach(stage::dependsOn);
|
.forEach(stage::dependsOn);
|
||||||
try {
|
CompletionStages.join(stage.freeze());
|
||||||
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;
|
|
||||||
tasks.clear();
|
tasks.clear();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void setRollbackOnly() {
|
protected void rollbackImpl() {
|
||||||
rollback = true;
|
tasks.clear();
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean getRollbackOnly() {
|
|
||||||
return rollback;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean isActive() {
|
|
||||||
return active;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void put(K key, V value, long lifespan, TimeUnit timeUnit) {
|
public void put(K key, V value, long lifespan, TimeUnit timeUnit) {
|
||||||
|
@ -126,6 +123,39 @@ public class RemoteInfinispanKeycloakTransaction<K, V> implements KeycloakTransa
|
||||||
return cache;
|
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) {
|
private CompletionStage<?> commitOperation(Operation<K, V> operation) {
|
||||||
try {
|
try {
|
||||||
return operation.execute(cache);
|
return operation.execute(cache);
|
||||||
|
|
|
@ -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;
|
package org.keycloak.models.sessions.infinispan.remote;
|
||||||
|
|
||||||
import org.infinispan.client.hotrod.Flag;
|
import org.infinispan.client.hotrod.Flag;
|
||||||
|
|
|
@ -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;
|
package org.keycloak.models.sessions.infinispan.remote;
|
||||||
|
|
||||||
|
import java.lang.invoke.MethodHandles;
|
||||||
|
|
||||||
import org.infinispan.client.hotrod.RemoteCache;
|
import org.infinispan.client.hotrod.RemoteCache;
|
||||||
import org.jboss.logging.Logger;
|
import org.jboss.logging.Logger;
|
||||||
import org.keycloak.Config;
|
import org.keycloak.Config;
|
||||||
|
@ -8,13 +27,12 @@ import org.keycloak.models.KeycloakSession;
|
||||||
import org.keycloak.models.KeycloakSessionFactory;
|
import org.keycloak.models.KeycloakSessionFactory;
|
||||||
import org.keycloak.models.SingleUseObjectProviderFactory;
|
import org.keycloak.models.SingleUseObjectProviderFactory;
|
||||||
import org.keycloak.models.sessions.infinispan.entities.SingleUseObjectValueEntity;
|
import org.keycloak.models.sessions.infinispan.entities.SingleUseObjectValueEntity;
|
||||||
|
import org.keycloak.provider.EnvironmentDependentProviderFactory;
|
||||||
import java.lang.invoke.MethodHandles;
|
|
||||||
|
|
||||||
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.ACTION_TOKEN_CACHE;
|
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.ACTION_TOKEN_CACHE;
|
||||||
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.getRemoteCache;
|
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());
|
private final static Logger logger = Logger.getLogger(MethodHandles.lookup().lookupClass());
|
||||||
|
|
||||||
|
|
|
@ -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;
|
package org.keycloak.models.sessions.infinispan.remote;
|
||||||
|
|
||||||
|
import java.lang.invoke.MethodHandles;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
import org.jboss.logging.Logger;
|
import org.jboss.logging.Logger;
|
||||||
import org.keycloak.Config;
|
import org.keycloak.Config;
|
||||||
import org.keycloak.connections.infinispan.InfinispanUtil;
|
import org.keycloak.connections.infinispan.InfinispanUtil;
|
||||||
import org.keycloak.infinispan.util.InfinispanUtils;
|
import org.keycloak.infinispan.util.InfinispanUtils;
|
||||||
import org.keycloak.models.KeycloakSession;
|
import org.keycloak.models.KeycloakSession;
|
||||||
import org.keycloak.models.KeycloakSessionFactory;
|
import org.keycloak.models.KeycloakSessionFactory;
|
||||||
|
import org.keycloak.provider.EnvironmentDependentProviderFactory;
|
||||||
import org.keycloak.provider.ProviderConfigProperty;
|
import org.keycloak.provider.ProviderConfigProperty;
|
||||||
import org.keycloak.provider.ProviderConfigurationBuilder;
|
import org.keycloak.provider.ProviderConfigurationBuilder;
|
||||||
import org.keycloak.sessions.StickySessionEncoderProvider;
|
import org.keycloak.sessions.StickySessionEncoderProvider;
|
||||||
import org.keycloak.sessions.StickySessionEncoderProviderFactory;
|
import org.keycloak.sessions.StickySessionEncoderProviderFactory;
|
||||||
|
|
||||||
import java.lang.invoke.MethodHandles;
|
public class RemoteStickySessionEncoderProviderFactory implements StickySessionEncoderProviderFactory, EnvironmentDependentProviderFactory {
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
public class RemoteStickySessionEncoderProviderFactory implements StickySessionEncoderProviderFactory {
|
|
||||||
|
|
||||||
private static final Logger log = Logger.getLogger(MethodHandles.lookup().lookupClass());
|
private static final Logger log = Logger.getLogger(MethodHandles.lookup().lookupClass());
|
||||||
private static final char SEPARATOR = '.';
|
private static final char SEPARATOR = '.';
|
||||||
|
|
|
@ -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() {
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
|
@ -35,7 +35,9 @@ public class SessionTimeouts {
|
||||||
/**
|
/**
|
||||||
* This indicates that entry is already expired and should be removed from the cache
|
* 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.
|
* Get the maximum lifespan, which this userSession can remain in the infinispan cache.
|
||||||
|
@ -211,7 +213,7 @@ public class SessionTimeouts {
|
||||||
* @return
|
* @return
|
||||||
*/
|
*/
|
||||||
public static long getLoginFailuresLifespanMs(RealmModel realm, ClientModel client, LoginFailureEntity loginFailureEntity) {
|
public static long getLoginFailuresLifespanMs(RealmModel realm, ClientModel client, LoginFailureEntity loginFailureEntity) {
|
||||||
return -1l;
|
return IMMORTAL_FLAG;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -224,6 +226,6 @@ public class SessionTimeouts {
|
||||||
* @return
|
* @return
|
||||||
*/
|
*/
|
||||||
public static long getLoginFailuresMaxIdleMs(RealmModel realm, ClientModel client, LoginFailureEntity loginFailureEntity) {
|
public static long getLoginFailuresMaxIdleMs(RealmModel realm, ClientModel client, LoginFailureEntity loginFailureEntity) {
|
||||||
return -1l;
|
return IMMORTAL_FLAG;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,3 +16,4 @@
|
||||||
#
|
#
|
||||||
|
|
||||||
org.keycloak.models.sessions.infinispan.InfinispanUserLoginFailureProviderFactory
|
org.keycloak.models.sessions.infinispan.InfinispanUserLoginFailureProviderFactory
|
||||||
|
org.keycloak.models.sessions.infinispan.remote.RemoteUserLoginFailureProviderFactory
|
|
@ -17,11 +17,10 @@
|
||||||
|
|
||||||
package org.keycloak.cluster;
|
package org.keycloak.cluster;
|
||||||
|
|
||||||
import org.keycloak.provider.EnvironmentDependentProviderFactory;
|
|
||||||
import org.keycloak.provider.ProviderFactory;
|
import org.keycloak.provider.ProviderFactory;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
|
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
|
||||||
*/
|
*/
|
||||||
public interface ClusterProviderFactory extends ProviderFactory<ClusterProvider>, EnvironmentDependentProviderFactory {
|
public interface ClusterProviderFactory extends ProviderFactory<ClusterProvider> {
|
||||||
}
|
}
|
||||||
|
|
|
@ -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.AUTHENTICATION_SESSIONS_CACHE_NAME;
|
||||||
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.CLIENT_SESSION_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.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_CLIENT_SESSION_CACHE_NAME;
|
||||||
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.OFFLINE_USER_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;
|
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.USER_SESSION_CACHE_NAME;
|
||||||
|
@ -125,6 +126,7 @@ public class CacheManagerFactory {
|
||||||
}
|
}
|
||||||
|
|
||||||
private RemoteCacheManager startRemoteCacheManager() {
|
private RemoteCacheManager startRemoteCacheManager() {
|
||||||
|
logger.info("Starting Infinispan remote cache manager (Hot Rod Client)");
|
||||||
String cacheRemoteHost = requiredStringProperty(CACHE_REMOTE_HOST_PROPERTY);
|
String cacheRemoteHost = requiredStringProperty(CACHE_REMOTE_HOST_PROPERTY);
|
||||||
Integer cacheRemotePort = Configuration.getOptionalKcValue(CACHE_REMOTE_PORT_PROPERTY)
|
Integer cacheRemotePort = Configuration.getOptionalKcValue(CACHE_REMOTE_PORT_PROPERTY)
|
||||||
.map(Integer::parseInt)
|
.map(Integer::parseInt)
|
||||||
|
@ -172,6 +174,7 @@ public class CacheManagerFactory {
|
||||||
}
|
}
|
||||||
|
|
||||||
private CompletableFuture<DefaultCacheManager> startEmbeddedCacheManager(String config) {
|
private CompletableFuture<DefaultCacheManager> startEmbeddedCacheManager(String config) {
|
||||||
|
logger.info("Starting Infinispan embedded cache manager");
|
||||||
ConfigurationBuilderHolder builder = new ParserRegistry().parse(config);
|
ConfigurationBuilderHolder builder = new ParserRegistry().parse(config);
|
||||||
|
|
||||||
if (builder.getNamedConfigurationBuilders().entrySet().stream().anyMatch(c -> c.getValue().clustering().cacheMode().isClustered())) {
|
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(WORK_CACHE_NAME);
|
||||||
builders.remove(AUTHENTICATION_SESSIONS_CACHE_NAME);
|
builders.remove(AUTHENTICATION_SESSIONS_CACHE_NAME);
|
||||||
builders.remove(ACTION_TOKEN_CACHE);
|
builders.remove(ACTION_TOKEN_CACHE);
|
||||||
|
builders.remove(LOGIN_FAILURE_CACHE_NAME);
|
||||||
}
|
}
|
||||||
|
|
||||||
var start = isStartEagerly();
|
var start = isStartEagerly();
|
||||||
|
@ -232,7 +236,7 @@ public class CacheManagerFactory {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static boolean isRemoteTLSEnabled() {
|
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() {
|
private static boolean isRemoteAuthenticationEnabled() {
|
||||||
|
@ -241,7 +245,7 @@ public class CacheManagerFactory {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static boolean createRemoteCaches() {
|
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() {
|
private static SSLContext createSSLContext() {
|
||||||
|
|
|
@ -17,11 +17,10 @@
|
||||||
|
|
||||||
package org.keycloak.models;
|
package org.keycloak.models;
|
||||||
|
|
||||||
import org.keycloak.provider.EnvironmentDependentProviderFactory;
|
|
||||||
import org.keycloak.provider.ProviderFactory;
|
import org.keycloak.provider.ProviderFactory;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
|
* @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> {
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,11 +17,10 @@
|
||||||
|
|
||||||
package org.keycloak.sessions;
|
package org.keycloak.sessions;
|
||||||
|
|
||||||
import org.keycloak.provider.EnvironmentDependentProviderFactory;
|
|
||||||
import org.keycloak.provider.ProviderFactory;
|
import org.keycloak.provider.ProviderFactory;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
|
* @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> {
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,13 +17,12 @@
|
||||||
|
|
||||||
package org.keycloak.sessions;
|
package org.keycloak.sessions;
|
||||||
|
|
||||||
import org.keycloak.provider.EnvironmentDependentProviderFactory;
|
|
||||||
import org.keycloak.provider.ProviderFactory;
|
import org.keycloak.provider.ProviderFactory;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
|
* @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
|
* For testing purpose only
|
||||||
|
|
|
@ -66,6 +66,7 @@
|
||||||
<migration.73.version>4.8.3.Final</migration.73.version>
|
<migration.73.version>4.8.3.Final</migration.73.version>
|
||||||
|
|
||||||
<!-- By default, skip docker-maven-plugin when running base tests-->
|
<!-- 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.skip>true</docker.database.skip>
|
||||||
<docker.database.postStart>/bin/true</docker.database.postStart>
|
<docker.database.postStart>/bin/true</docker.database.postStart>
|
||||||
<docker.database.wait-for-log-regex>NEVER-MATCHING-REGEX</docker.database.wait-for-log-regex>
|
<docker.database.wait-for-log-regex>NEVER-MATCHING-REGEX</docker.database.wait-for-log-regex>
|
||||||
|
|
|
@ -384,9 +384,6 @@
|
||||||
<groupId>io.fabric8</groupId>
|
<groupId>io.fabric8</groupId>
|
||||||
<artifactId>docker-maven-plugin</artifactId>
|
<artifactId>docker-maven-plugin</artifactId>
|
||||||
<version>${docker.maven.plugin.version}</version>
|
<version>${docker.maven.plugin.version}</version>
|
||||||
<configuration>
|
|
||||||
<skip>${docker.database.skip}</skip>
|
|
||||||
</configuration>
|
|
||||||
<executions>
|
<executions>
|
||||||
<execution>
|
<execution>
|
||||||
<id>start-db-container</id>
|
<id>start-db-container</id>
|
||||||
|
@ -395,6 +392,7 @@
|
||||||
<goal>start</goal>
|
<goal>start</goal>
|
||||||
</goals>
|
</goals>
|
||||||
<configuration>
|
<configuration>
|
||||||
|
<skip>${docker.database.skip}</skip>
|
||||||
<showLogs>true</showLogs>
|
<showLogs>true</showLogs>
|
||||||
<images>
|
<images>
|
||||||
<image>
|
<image>
|
||||||
|
@ -447,6 +445,9 @@
|
||||||
<goals>
|
<goals>
|
||||||
<goal>stop</goal>
|
<goal>stop</goal>
|
||||||
</goals>
|
</goals>
|
||||||
|
<configuration>
|
||||||
|
<skip>${docker.database.skip}</skip>
|
||||||
|
</configuration>
|
||||||
</execution>
|
</execution>
|
||||||
</executions>
|
</executions>
|
||||||
</plugin>
|
</plugin>
|
||||||
|
@ -765,6 +766,71 @@
|
||||||
</dependencies>
|
</dependencies>
|
||||||
</profile>
|
</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>
|
</profiles>
|
||||||
|
|
||||||
</project>
|
</project>
|
||||||
|
|
|
@ -29,6 +29,8 @@ import java.security.KeyManagementException;
|
||||||
import java.security.NoSuchAlgorithmException;
|
import java.security.NoSuchAlgorithmException;
|
||||||
import java.security.cert.X509Certificate;
|
import java.security.cert.X509Certificate;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
@ -37,6 +39,8 @@ import java.util.Map;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import java.util.function.Supplier;
|
import java.util.function.Supplier;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import javax.net.ssl.HostnameVerifier;
|
import javax.net.ssl.HostnameVerifier;
|
||||||
import javax.net.ssl.HttpsURLConnection;
|
import javax.net.ssl.HttpsURLConnection;
|
||||||
import javax.net.ssl.SSLContext;
|
import javax.net.ssl.SSLContext;
|
||||||
|
@ -207,6 +211,16 @@ public abstract class AbstractQuarkusDeployableContainer implements DeployableCo
|
||||||
addStorageOptions(storeProvider, commands);
|
addStorageOptions(storeProvider, commands);
|
||||||
addFeaturesOption(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;
|
return commands;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -407,4 +421,12 @@ public abstract class AbstractQuarkusDeployableContainer implements DeployableCo
|
||||||
|
|
||||||
configuration.appendJavaOpts("-Djava.security.properties=" + System.getProperty("auth.server.java.security.file"));
|
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());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,6 +22,7 @@ import org.jboss.resteasy.client.jaxrs.ResteasyClient;
|
||||||
import org.jboss.resteasy.client.jaxrs.ResteasyClientBuilder;
|
import org.jboss.resteasy.client.jaxrs.ResteasyClientBuilder;
|
||||||
import org.jboss.resteasy.client.jaxrs.ResteasyWebTarget;
|
import org.jboss.resteasy.client.jaxrs.ResteasyWebTarget;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
import org.junit.AssumptionViolatedException;
|
||||||
import org.keycloak.common.Profile;
|
import org.keycloak.common.Profile;
|
||||||
import org.keycloak.testsuite.ProfileAssume;
|
import org.keycloak.testsuite.ProfileAssume;
|
||||||
import org.keycloak.testsuite.client.resources.TestApplicationResource;
|
import org.keycloak.testsuite.client.resources.TestApplicationResource;
|
||||||
|
@ -185,6 +186,8 @@ public class KeycloakTestingClient implements AutoCloseable {
|
||||||
|
|
||||||
if (t instanceof AssertionError) {
|
if (t instanceof AssertionError) {
|
||||||
throw (AssertionError) t;
|
throw (AssertionError) t;
|
||||||
|
} else if (t instanceof AssumptionViolatedException) {
|
||||||
|
throw (AssumptionViolatedException) t;
|
||||||
} else {
|
} else {
|
||||||
throw new RunOnServerException(t);
|
throw new RunOnServerException(t);
|
||||||
}
|
}
|
||||||
|
|
|
@ -78,6 +78,7 @@ import java.util.Calendar;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
import java.util.Scanner;
|
import java.util.Scanner;
|
||||||
import java.util.concurrent.Callable;
|
import java.util.concurrent.Callable;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
|
@ -86,7 +87,9 @@ import java.util.concurrent.Future;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
import java.util.function.Consumer;
|
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.MatcherAssert.assertThat;
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
import static org.hamcrest.Matchers.is;
|
import static org.hamcrest.Matchers.is;
|
||||||
|
@ -159,6 +162,7 @@ public abstract class AbstractKeycloakTest {
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void beforeAbstractKeycloakTest() throws Exception {
|
public void beforeAbstractKeycloakTest() throws Exception {
|
||||||
|
ProfileAssume.setTestContext(testContext);
|
||||||
adminClient = testContext.getAdminClient();
|
adminClient = testContext.getAdminClient();
|
||||||
if (adminClient == null || adminClient.isClosed()) {
|
if (adminClient == null || adminClient.isClosed()) {
|
||||||
reconnectAdminClient();
|
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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,12 +17,16 @@
|
||||||
|
|
||||||
package org.keycloak.testsuite.model;
|
package org.keycloak.testsuite.model;
|
||||||
|
|
||||||
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
import org.keycloak.common.Profile;
|
||||||
import org.keycloak.common.util.Time;
|
import org.keycloak.common.util.Time;
|
||||||
import org.keycloak.models.ClientModel;
|
import org.keycloak.models.ClientModel;
|
||||||
|
import org.keycloak.models.Constants;
|
||||||
import org.keycloak.models.KeycloakSession;
|
import org.keycloak.models.KeycloakSession;
|
||||||
import org.keycloak.models.RealmModel;
|
import org.keycloak.models.RealmModel;
|
||||||
import org.keycloak.models.UserManager;
|
import org.keycloak.models.UserManager;
|
||||||
|
@ -37,15 +41,13 @@ import org.keycloak.sessions.CommonClientSessionModel;
|
||||||
import org.keycloak.sessions.RootAuthenticationSessionModel;
|
import org.keycloak.sessions.RootAuthenticationSessionModel;
|
||||||
import org.keycloak.testsuite.AbstractTestRealmKeycloakTest;
|
import org.keycloak.testsuite.AbstractTestRealmKeycloakTest;
|
||||||
import org.keycloak.testsuite.arquillian.annotation.ModelTest;
|
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.Is.is;
|
||||||
import static org.hamcrest.core.IsNull.notNullValue;
|
import static org.hamcrest.core.IsNull.notNullValue;
|
||||||
import static org.hamcrest.core.IsNull.nullValue;
|
import static org.hamcrest.core.IsNull.nullValue;
|
||||||
import static org.hamcrest.MatcherAssert.assertThat;
|
import static org.junit.Assume.assumeFalse;
|
||||||
import org.keycloak.models.Constants;
|
|
||||||
import org.keycloak.testsuite.util.InfinispanTestTimeServiceRule;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
|
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
|
||||||
|
@ -212,10 +214,10 @@ public class AuthenticationSessionProviderTest extends AbstractTestRealmKeycloak
|
||||||
@Test
|
@Test
|
||||||
@ModelTest
|
@ModelTest
|
||||||
public void testExpiredAuthSessions(KeycloakSession session) {
|
public void testExpiredAuthSessions(KeycloakSession session) {
|
||||||
|
assumeFalse(Profile.isFeatureEnabled(Profile.Feature.REMOTE_CACHE));
|
||||||
AtomicReference<String> authSessionID = new AtomicReference<>();
|
AtomicReference<String> authSessionID = new AtomicReference<>();
|
||||||
|
|
||||||
KeycloakModelUtils.runJobInTransaction(session.getKeycloakSessionFactory(), (KeycloakSession sessionExpired) -> {
|
KeycloakModelUtils.runJobInTransaction(session.getKeycloakSessionFactory(), mainSession -> {
|
||||||
KeycloakSession mainSession = sessionExpired;
|
|
||||||
try {
|
try {
|
||||||
// AccessCodeLifespan = 10 ; AccessCodeLifespanUserAction = 10 ; AccessCodeLifespanLogin = 30
|
// AccessCodeLifespan = 10 ; AccessCodeLifespanUserAction = 10 ; AccessCodeLifespanLogin = 30
|
||||||
setAccessCodeLifespan(mainSession, 10, 10, 30);
|
setAccessCodeLifespan(mainSession, 10, 10, 30);
|
||||||
|
|
|
@ -23,6 +23,7 @@ import org.junit.Before;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.keycloak.common.util.Time;
|
import org.keycloak.common.util.Time;
|
||||||
|
import org.keycloak.infinispan.util.InfinispanUtils;
|
||||||
import org.keycloak.models.AuthenticatedClientSessionModel;
|
import org.keycloak.models.AuthenticatedClientSessionModel;
|
||||||
import org.keycloak.models.ClientModel;
|
import org.keycloak.models.ClientModel;
|
||||||
import org.keycloak.models.KeycloakSession;
|
import org.keycloak.models.KeycloakSession;
|
||||||
|
@ -322,14 +323,20 @@ public class UserSessionProviderTest extends AbstractTestRealmKeycloakTest {
|
||||||
RealmModel realm = session.realms().getRealmByName("test");
|
RealmModel realm = session.realms().getRealmByName("test");
|
||||||
createSessions(session);
|
createSessions(session);
|
||||||
|
|
||||||
KeycloakModelUtils.runJobInTransaction(session.getKeycloakSessionFactory(), (KeycloakSession kcSession) -> {
|
KeycloakModelUtils.runJobInTransaction(session.getKeycloakSessionFactory(), kcSession -> kcSession.sessions().removeUserSessions(realm));
|
||||||
kcSession.sessions().removeUserSessions(realm);
|
|
||||||
});
|
|
||||||
|
|
||||||
assertEquals(0, session.sessions().getUserSessionsStream(realm, session.users().getUserByUsername(realm, "user1"))
|
var user1 = session.users().getUserByUsername(realm, "user1");
|
||||||
.count());
|
var user2 = session.users().getUserByUsername(realm, "user2");
|
||||||
assertEquals(0, session.sessions().getUserSessionsStream(realm, session.users().getUserByUsername(realm, "user2"))
|
|
||||||
.count());
|
// 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
|
@Test
|
||||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.http.impl.client.CloseableHttpClient;
|
||||||
import org.apache.http.impl.client.HttpClientBuilder;
|
import org.apache.http.impl.client.HttpClientBuilder;
|
||||||
import org.apache.http.message.BasicNameValuePair;
|
import org.apache.http.message.BasicNameValuePair;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
import org.junit.Assume;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
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.ClientsResource;
|
||||||
import org.keycloak.admin.client.resource.RealmResource;
|
import org.keycloak.admin.client.resource.RealmResource;
|
||||||
import org.keycloak.admin.client.resource.UserResource;
|
import org.keycloak.admin.client.resource.UserResource;
|
||||||
|
import org.keycloak.common.Profile;
|
||||||
import org.keycloak.common.enums.SslRequired;
|
import org.keycloak.common.enums.SslRequired;
|
||||||
import org.keycloak.common.util.Base64Url;
|
import org.keycloak.common.util.Base64Url;
|
||||||
import org.keycloak.crypto.Algorithm;
|
import org.keycloak.crypto.Algorithm;
|
||||||
|
@ -44,6 +46,7 @@ import org.keycloak.crypto.ECDSAAlgorithm;
|
||||||
import org.keycloak.crypto.KeyUse;
|
import org.keycloak.crypto.KeyUse;
|
||||||
import org.keycloak.events.Details;
|
import org.keycloak.events.Details;
|
||||||
import org.keycloak.events.Errors;
|
import org.keycloak.events.Errors;
|
||||||
|
import org.keycloak.infinispan.util.InfinispanUtils;
|
||||||
import org.keycloak.jose.jwk.JWK;
|
import org.keycloak.jose.jwk.JWK;
|
||||||
import org.keycloak.jose.jws.JWSHeader;
|
import org.keycloak.jose.jws.JWSHeader;
|
||||||
import org.keycloak.jose.jws.JWSInput;
|
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.AbstractKeycloakTest;
|
||||||
import org.keycloak.testsuite.ActionURIUtils;
|
import org.keycloak.testsuite.ActionURIUtils;
|
||||||
import org.keycloak.testsuite.AssertEvents;
|
import org.keycloak.testsuite.AssertEvents;
|
||||||
|
import org.keycloak.testsuite.ProfileAssume;
|
||||||
import org.keycloak.testsuite.admin.ApiUtil;
|
import org.keycloak.testsuite.admin.ApiUtil;
|
||||||
import org.keycloak.testsuite.util.AdminClientUtil;
|
import org.keycloak.testsuite.util.AdminClientUtil;
|
||||||
import org.keycloak.testsuite.util.ClientBuilder;
|
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.assertNotNull;
|
||||||
import static org.junit.Assert.assertNull;
|
import static org.junit.Assert.assertNull;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
import static org.junit.Assume.*;
|
||||||
import static org.keycloak.testsuite.Assert.assertExpiration;
|
import static org.keycloak.testsuite.Assert.assertExpiration;
|
||||||
import static org.keycloak.testsuite.admin.AbstractAdminTest.loadJson;
|
import static org.keycloak.testsuite.admin.AbstractAdminTest.loadJson;
|
||||||
import static org.keycloak.testsuite.admin.ApiUtil.findClientByClientId;
|
import static org.keycloak.testsuite.admin.ApiUtil.findClientByClientId;
|
||||||
|
@ -383,6 +388,7 @@ public class AccessTokenTest extends AbstractKeycloakTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void accessTokenCodeExpired() {
|
public void accessTokenCodeExpired() {
|
||||||
|
ProfileAssume.assumeFeatureDisabled(Profile.Feature.REMOTE_CACHE);
|
||||||
getTestingClient().testing().setTestingInfinispanTimeService();
|
getTestingClient().testing().setTestingInfinispanTimeService();
|
||||||
RealmManager.realm(adminClient.realm("test")).accessCodeLifeSpan(1);
|
RealmManager.realm(adminClient.realm("test")).accessCodeLifeSpan(1);
|
||||||
oauth.doLogin("test-user@localhost", "password");
|
oauth.doLogin("test-user@localhost", "password");
|
||||||
|
|
|
@ -15,9 +15,9 @@
|
||||||
<packaging>jar</packaging>
|
<packaging>jar</packaging>
|
||||||
|
|
||||||
<properties>
|
<properties>
|
||||||
<maven.compiler.release>11</maven.compiler.release>
|
<maven.compiler.release>17</maven.compiler.release>
|
||||||
<maven.compiler.source>11</maven.compiler.source>
|
<maven.compiler.source>17</maven.compiler.source>
|
||||||
<maven.compiler.target>11</maven.compiler.target>
|
<maven.compiler.target>17</maven.compiler.target>
|
||||||
|
|
||||||
<keycloak.connectionsJpa.driver>org.h2.Driver</keycloak.connectionsJpa.driver>
|
<keycloak.connectionsJpa.driver>org.h2.Driver</keycloak.connectionsJpa.driver>
|
||||||
<keycloak.connectionsJpa.database>keycloak</keycloak.connectionsJpa.database>
|
<keycloak.connectionsJpa.database>keycloak</keycloak.connectionsJpa.database>
|
||||||
|
@ -224,6 +224,19 @@
|
||||||
<properties>
|
<properties>
|
||||||
<keycloak.model.parameters>CrossDCInfinispan,Jpa,PersistentUserSessions</keycloak.model.parameters>
|
<keycloak.model.parameters>CrossDCInfinispan,Jpa,PersistentUserSessions</keycloak.model.parameters>
|
||||||
</properties>
|
</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>
|
||||||
|
|
||||||
<profile>
|
<profile>
|
||||||
|
|
|
@ -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;
|
package org.keycloak.testsuite.model.infinispan;
|
||||||
|
|
||||||
import java.util.Arrays;
|
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.AUTHENTICATION_SESSIONS_CACHE_NAME;
|
||||||
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.CLUSTERED_CACHE_NAMES;
|
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.LOCAL_CACHE_NAMES;
|
||||||
|
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.LOGIN_FAILURE_CACHE_NAME;
|
||||||
import static org.keycloak.connections.infinispan.InfinispanConnectionProvider.WORK_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, WORK_CACHE_NAME);
|
||||||
assertEmbeddedCacheDoesNotExists(clusterProvider, AUTHENTICATION_SESSIONS_CACHE_NAME);
|
assertEmbeddedCacheDoesNotExists(clusterProvider, AUTHENTICATION_SESSIONS_CACHE_NAME);
|
||||||
assertEmbeddedCacheDoesNotExists(clusterProvider, ACTION_TOKEN_CACHE);
|
assertEmbeddedCacheDoesNotExists(clusterProvider, ACTION_TOKEN_CACHE);
|
||||||
|
assertEmbeddedCacheDoesNotExists(clusterProvider, LOGIN_FAILURE_CACHE_NAME);
|
||||||
|
|
||||||
// TODO [pruivo] all caches eventually won't exists in embedded
|
// TODO [pruivo] all caches eventually won't exists in embedded
|
||||||
Arrays.stream(CLUSTERED_CACHE_NAMES)
|
Arrays.stream(CLUSTERED_CACHE_NAMES)
|
||||||
.filter(Predicate.not(Predicate.isEqual(WORK_CACHE_NAME)))
|
.filter(Predicate.not(Predicate.isEqual(WORK_CACHE_NAME)))
|
||||||
.filter(Predicate.not(Predicate.isEqual(AUTHENTICATION_SESSIONS_CACHE_NAME)))
|
.filter(Predicate.not(Predicate.isEqual(AUTHENTICATION_SESSIONS_CACHE_NAME)))
|
||||||
.filter(Predicate.not(Predicate.isEqual(ACTION_TOKEN_CACHE)))
|
.filter(Predicate.not(Predicate.isEqual(ACTION_TOKEN_CACHE)))
|
||||||
|
.filter(Predicate.not(Predicate.isEqual(LOGIN_FAILURE_CACHE_NAME)))
|
||||||
.forEach(s -> assertEmbeddedCacheExists(clusterProvider, s));
|
.forEach(s -> assertEmbeddedCacheExists(clusterProvider, s));
|
||||||
|
|
||||||
Arrays.stream(CLUSTERED_CACHE_NAMES).forEach(s -> assertRemoteCacheExists(clusterProvider, s));
|
Arrays.stream(CLUSTERED_CACHE_NAMES).forEach(s -> assertRemoteCacheExists(clusterProvider, s));
|
||||||
|
|
|
@ -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);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -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.
|
* and other contributors as indicated by the @author tags.
|
||||||
*
|
*
|
||||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||||
|
@ -16,6 +16,10 @@
|
||||||
*/
|
*/
|
||||||
package org.keycloak.testsuite.model.parameters;
|
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 com.google.common.collect.ImmutableSet;
|
||||||
import org.junit.runner.Description;
|
import org.junit.runner.Description;
|
||||||
import org.junit.runners.model.Statement;
|
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.RemoteInfinispanAuthenticationSessionProviderFactory;
|
||||||
import org.keycloak.models.sessions.infinispan.remote.RemoteInfinispanSingleUseObjectProviderFactory;
|
import org.keycloak.models.sessions.infinispan.remote.RemoteInfinispanSingleUseObjectProviderFactory;
|
||||||
import org.keycloak.models.sessions.infinispan.remote.RemoteStickySessionEncoderProviderFactory;
|
import org.keycloak.models.sessions.infinispan.remote.RemoteStickySessionEncoderProviderFactory;
|
||||||
|
import org.keycloak.models.sessions.infinispan.remote.RemoteUserLoginFailureProviderFactory;
|
||||||
import org.keycloak.provider.ProviderFactory;
|
import org.keycloak.provider.ProviderFactory;
|
||||||
import org.keycloak.testsuite.model.Config;
|
import org.keycloak.testsuite.model.Config;
|
||||||
import org.keycloak.testsuite.model.HotRodServerRule;
|
import org.keycloak.testsuite.model.HotRodServerRule;
|
||||||
import org.keycloak.testsuite.model.KeycloakModelParameters;
|
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}.
|
* Copied from {@link CrossDCInfinispan}.
|
||||||
* <p>
|
* <p>
|
||||||
|
@ -59,6 +60,7 @@ public class RemoteInfinispan extends KeycloakModelParameters {
|
||||||
.add(RemoteInfinispanSingleUseObjectProviderFactory.class)
|
.add(RemoteInfinispanSingleUseObjectProviderFactory.class)
|
||||||
.add(RemoteStickySessionEncoderProviderFactory.class)
|
.add(RemoteStickySessionEncoderProviderFactory.class)
|
||||||
.add(RemoteLoadBalancerCheckProviderFactory.class)
|
.add(RemoteLoadBalancerCheckProviderFactory.class)
|
||||||
|
.add(RemoteUserLoginFailureProviderFactory.class)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -84,29 +84,9 @@ public class SessionTimeoutsTest extends KeycloakModelTest {
|
||||||
s.sessions().getOfflineUserSessionsStream(realm, user1).forEach(us -> s.sessions().removeOfflineUserSession(realm, us));
|
s.sessions().getOfflineUserSessionsStream(realm, user1).forEach(us -> s.sessions().removeOfflineUserSession(realm, us));
|
||||||
s.realms().removeRealm(realmId);
|
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);
|
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) {
|
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",
|
UserSessionModel userSession = session.sessions().createUserSession(UUID.randomUUID().toString(), realm, user, "user1", "127.0.0.1",
|
||||||
"form", true, null, null, UserSessionModel.SessionPersistenceState.PERSISTENT);
|
"form", true, null, null, UserSessionModel.SessionPersistenceState.PERSISTENT);
|
||||||
|
@ -345,6 +325,7 @@ public class SessionTimeoutsTest extends KeycloakModelTest {
|
||||||
Assert.assertNull(getUserSession(session, realm, sessions[0], offline));
|
Assert.assertNull(getUserSession(session, realm, sessions[0], offline));
|
||||||
return null;
|
return null;
|
||||||
});
|
});
|
||||||
|
processExpiration(offline);
|
||||||
} finally {
|
} finally {
|
||||||
setTimeOffset(0);
|
setTimeOffset(0);
|
||||||
}
|
}
|
||||||
|
@ -370,12 +351,12 @@ public class SessionTimeoutsTest extends KeycloakModelTest {
|
||||||
testUserClientMaxLifespanSmallerThanSession(true, true);
|
testUserClientMaxLifespanSmallerThanSession(true, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 10 * 1000)
|
@Test
|
||||||
public void testOfflineUserClientIdleTimeoutSmallerThanSessionNoRefresh() {
|
public void testOfflineUserClientIdleTimeoutSmallerThanSessionNoRefresh() {
|
||||||
testUserClientIdleTimeoutSmallerThanSession(0, true, false);
|
testUserClientIdleTimeoutSmallerThanSession(0, true, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 10 * 1000)
|
@Test
|
||||||
public void testOfflineUserClientIdleTimeoutSmallerThanSessionOneRefresh() {
|
public void testOfflineUserClientIdleTimeoutSmallerThanSessionOneRefresh() {
|
||||||
testUserClientIdleTimeoutSmallerThanSession(1, true, false);
|
testUserClientIdleTimeoutSmallerThanSession(1, true, false);
|
||||||
}
|
}
|
||||||
|
@ -400,12 +381,12 @@ public class SessionTimeoutsTest extends KeycloakModelTest {
|
||||||
testUserClientMaxLifespanSmallerThanSession(false, true);
|
testUserClientMaxLifespanSmallerThanSession(false, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 10 * 1000)
|
@Test
|
||||||
public void testOnlineUserClientIdleTimeoutSmallerThanSessionNoRefresh() {
|
public void testOnlineUserClientIdleTimeoutSmallerThanSessionNoRefresh() {
|
||||||
testUserClientIdleTimeoutSmallerThanSession(0, false, false);
|
testUserClientIdleTimeoutSmallerThanSession(0, false, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 10 * 1000)
|
@Test
|
||||||
public void testOnlineUserClientIdleTimeoutSmallerThanSessionOneRefresh() {
|
public void testOnlineUserClientIdleTimeoutSmallerThanSessionOneRefresh() {
|
||||||
testUserClientIdleTimeoutSmallerThanSession(1, false, false);
|
testUserClientIdleTimeoutSmallerThanSession(1, false, false);
|
||||||
}
|
}
|
||||||
|
@ -416,12 +397,26 @@ public class SessionTimeoutsTest extends KeycloakModelTest {
|
||||||
* @param offline boolean Indicates where we work with offline sessions
|
* @param offline boolean Indicates where we work with offline sessions
|
||||||
*/
|
*/
|
||||||
private void allowXSiteReplication(boolean offline) {
|
private void allowXSiteReplication(boolean offline) {
|
||||||
HotRodServerRule hotRodServer = getParameters(HotRodServerRule.class).findFirst().orElse(null);
|
var hotRodServer = getParameters(HotRodServerRule.class).findFirst();
|
||||||
if (hotRodServer != null) {
|
if (hotRodServer.isEmpty()) {
|
||||||
var cacheName = offline ? InfinispanConnectionProvider.OFFLINE_CLIENT_SESSION_CACHE_NAME : InfinispanConnectionProvider.CLIENT_SESSION_CACHE_NAME;
|
return;
|
||||||
var cache1 = hotRodServer.getHotRodCacheManager().getCache(cacheName);
|
|
||||||
var cache2 = hotRodServer.getHotRodCacheManager2().getCache(cacheName);
|
|
||||||
eventually(null, () -> cache1.size() == cache2.size(), 10000, 10, TimeUnit.MILLISECONDS);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
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();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in a new issue