Fix and correlate session timeout calculations in legacy and new map implementations

Closes https://github.com/keycloak/keycloak/issues/14854
Closes https://github.com/keycloak/keycloak/issues/11990
This commit is contained in:
rmartinc 2023-05-26 17:41:27 +02:00 committed by Marek Posolda
parent 8eee3f434b
commit 81aa588ddc
27 changed files with 1531 additions and 327 deletions

View file

@ -1166,6 +1166,9 @@ public class RealmCacheSession implements CacheRealmProvider {
StorageId storageId = new StorageId(cached.getId());
if (!storageId.isLocal()) {
ComponentModel component = realm.getComponent(storageId.getProviderId());
if (component == null) {
return null;
}
ClientStorageProviderModel model = new ClientStorageProviderModel(component);
// although we do set a timeout, Infinispan has no guarantees when the user will be evicted

View file

@ -316,6 +316,9 @@ public class UserCacheSession implements UserCache, OnCreateComponent, OnUpdateC
if (!storageId.isLocal()) {
ComponentModel component = realm.getComponent(storageId.getProviderId());
if (component == null) {
return null;
}
CacheableStorageProviderModel model = new CacheableStorageProviderModel(component);
// although we do set a timeout, Infinispan has no guarantees when the user will be evicted

View file

@ -24,6 +24,7 @@ import org.keycloak.Config;
import org.keycloak.cluster.ClusterProvider;
import org.keycloak.common.util.Time;
import org.keycloak.connections.infinispan.InfinispanConnectionProvider;
import org.keycloak.models.ClientModel;
import org.keycloak.models.KeycloakSession;
import org.keycloak.models.KeycloakSessionFactory;
import org.keycloak.models.KeycloakSessionTask;
@ -49,7 +50,6 @@ import org.keycloak.models.utils.PostMigrationEvent;
import java.io.Serializable;
import java.util.Set;
import java.util.function.BiFunction;
import static org.keycloak.models.sessions.infinispan.InfinispanAuthenticationSessionProviderFactory.PROVIDER_PRIORITY;
/**
@ -142,7 +142,7 @@ public class InfinispanUserLoginFailureProviderFactory implements UserLoginFailu
}
private <K, V extends SessionEntity> RemoteCache checkRemoteCache(KeycloakSession session, Cache<K, SessionEntityWrapper<V>> ispnCache, RemoteCacheInvoker.MaxIdleTimeLoader maxIdleLoader,
BiFunction<RealmModel, V, Long> lifespanMsLoader, BiFunction<RealmModel, V, Long> maxIdleTimeMsLoader) {
SessionFunction<V> lifespanMsLoader, SessionFunction<V> maxIdleTimeMsLoader) {
Set<RemoteStore> remoteStores = InfinispanUtil.getRemoteStores(ispnCache);
if (remoteStores.isEmpty()) {

View file

@ -27,7 +27,6 @@ import org.jboss.logging.Logger;
import org.keycloak.cluster.ClusterProvider;
import org.keycloak.common.util.Retry;
import org.keycloak.common.util.Time;
import org.keycloak.device.DeviceActivityManager;
import org.keycloak.models.AuthenticatedClientSessionModel;
import org.keycloak.models.ClientModel;
import org.keycloak.models.KeycloakSession;
@ -76,7 +75,6 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.BiFunction;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.function.Predicate;
@ -193,8 +191,13 @@ public class InfinispanUserSessionProvider implements UserSessionProvider {
final UUID clientSessionId = keyGenerator.generateKeyUUID(session, clientSessionCache);
AuthenticatedClientSessionEntity entity = new AuthenticatedClientSessionEntity(clientSessionId);
entity.setRealmId(realm.getId());
entity.setClientId(client.getId());
entity.setTimestamp(Time.currentTime());
entity.getNotes().put(AuthenticatedClientSessionModel.STARTED_AT_NOTE, String.valueOf(entity.getTimestamp()));
entity.getNotes().put(AuthenticatedClientSessionModel.USER_SESSION_STARTED_AT_NOTE, String.valueOf(userSession.getStarted()));
if (userSession.isRememberMe()) {
entity.getNotes().put(AuthenticatedClientSessionModel.USER_SESSION_REMEMBER_ME_NOTE, "true");
}
InfinispanChangelogBasedTransaction<String, UserSessionEntity> userSessionUpdateTx = getTransaction(false);
InfinispanChangelogBasedTransaction<UUID, AuthenticatedClientSessionEntity> clientSessionUpdateTx = getClientSessionTransaction(false);
@ -287,7 +290,12 @@ public class InfinispanUserSessionProvider implements UserSessionProvider {
return null;
}
return importUserSession(realm, offline, persistentUserSession);
UserSessionEntity sessionEntity = importUserSession(realm, offline, persistentUserSession);
if (sessionEntity == null) {
persister.removeUserSession(sessionId, offline);
}
return sessionEntity;
}
private UserSessionEntity getUserSessionEntityFromCacheOrImportIfNecessary(RealmModel realm, boolean offline, UserSessionModel persistentUserSession) {
@ -414,7 +422,13 @@ public class InfinispanUserSessionProvider implements UserSessionProvider {
return null;
}
return importClientSession((UserSessionAdapter) userSession, clientSession, getTransaction(offline), getClientSessionTransaction(offline), offline);
AuthenticatedClientSessionAdapter clientAdapter = importClientSession((UserSessionAdapter) userSession, clientSession, getTransaction(offline),
getClientSessionTransaction(offline), offline, true);
if (clientAdapter == null) {
persister.removeClientSession(userSession.getId(), client.getId(), offline);
}
return clientAdapter;
}
private AuthenticatedClientSessionEntity getClientSessionEntity(UUID id, boolean offline) {
@ -815,11 +829,12 @@ public class InfinispanUserSessionProvider implements UserSessionProvider {
InfinispanChangelogBasedTransaction<String, UserSessionEntity> userSessionUpdateTx = getTransaction(true);
InfinispanChangelogBasedTransaction<UUID, AuthenticatedClientSessionEntity> clientSessionUpdateTx = getClientSessionTransaction(true);
AuthenticatedClientSessionAdapter offlineClientSession = importClientSession(userSessionAdapter, clientSession, userSessionUpdateTx, clientSessionUpdateTx, true);
AuthenticatedClientSessionAdapter offlineClientSession = importClientSession(userSessionAdapter, clientSession, userSessionUpdateTx, clientSessionUpdateTx, true, false);
// update timestamp to current time
offlineClientSession.setTimestamp(Time.currentTime());
offlineClientSession.getNotes().put(AuthenticatedClientSessionModel.STARTED_AT_NOTE, String.valueOf(offlineClientSession.getTimestamp()));
offlineClientSession.getNotes().put(AuthenticatedClientSessionModel.USER_SESSION_STARTED_AT_NOTE, String.valueOf(offlineUserSession.getStarted()));
session.getProvider(UserSessionPersisterProvider.class).createClientSession(clientSession, true);
@ -863,7 +878,8 @@ public class InfinispanUserSessionProvider implements UserSessionProvider {
for (Map.Entry<String, AuthenticatedClientSessionModel> entry : persistentUserSession.getAuthenticatedClientSessions().entrySet()) {
String clientUUID = entry.getKey();
AuthenticatedClientSessionModel clientSession = entry.getValue();
AuthenticatedClientSessionEntity clientSessionToImport = createAuthenticatedClientSessionInstance(clientSession, userSessionEntityToImport.getRealmId(), offline);
AuthenticatedClientSessionEntity clientSessionToImport = createAuthenticatedClientSessionInstance(clientSession,
userSessionEntityToImport.getRealmId(), clientUUID, offline);
// Update timestamp to same value as userSession. LastSessionRefresh of userSession from DB will have correct value
clientSessionToImport.setTimestamp(userSessionEntityToImport.getLastSessionRefresh());
@ -970,14 +986,15 @@ public class InfinispanUserSessionProvider implements UserSessionProvider {
}
private <T extends SessionEntity> void importSessionsWithExpiration(Map<? extends Object, SessionEntityWrapper<T>> sessionsById,
BasicCache cache, BiFunction<RealmModel, T, Long> lifespanMsCalculator,
BiFunction<RealmModel, T, Long> maxIdleTimeMsCalculator) {
BasicCache cache, SessionFunction<T> lifespanMsCalculator,
SessionFunction<T> maxIdleTimeMsCalculator) {
sessionsById.forEach((id, sessionEntityWrapper) -> {
T sessionEntity = sessionEntityWrapper.getEntity();
RealmModel currentRealm = session.realms().getRealm(sessionEntity.getRealmId());
long lifespan = lifespanMsCalculator.apply(currentRealm, sessionEntity);
long maxIdle = maxIdleTimeMsCalculator.apply(currentRealm, sessionEntity);
ClientModel client = sessionEntityWrapper.getClientIfNeeded(currentRealm);
long lifespan = lifespanMsCalculator.apply(currentRealm, client, sessionEntity);
long maxIdle = maxIdleTimeMsCalculator.apply(currentRealm, client, sessionEntity);
if(lifespan != SessionTimeouts.ENTRY_EXPIRED_FLAG
&& maxIdle != SessionTimeouts.ENTRY_EXPIRED_FLAG ) {
@ -1055,8 +1072,21 @@ public class InfinispanUserSessionProvider implements UserSessionProvider {
private AuthenticatedClientSessionAdapter importClientSession(UserSessionAdapter sessionToImportInto, AuthenticatedClientSessionModel clientSession,
InfinispanChangelogBasedTransaction<String, UserSessionEntity> userSessionUpdateTx,
InfinispanChangelogBasedTransaction<UUID, AuthenticatedClientSessionEntity> clientSessionUpdateTx,
boolean offline) {
AuthenticatedClientSessionEntity entity = createAuthenticatedClientSessionInstance(clientSession, sessionToImportInto.getRealm().getId(), offline);
boolean offline, boolean checkExpiration) {
AuthenticatedClientSessionEntity entity = createAuthenticatedClientSessionInstance(clientSession,
sessionToImportInto.getRealm().getId(), clientSession.getClient().getId(), offline);
if (checkExpiration) {
SessionFunction<AuthenticatedClientSessionEntity> lifespanChecker = offline
? SessionTimeouts::getOfflineClientSessionLifespanMs : SessionTimeouts::getClientSessionLifespanMs;
SessionFunction<AuthenticatedClientSessionEntity> idleTimeoutChecker = offline
? SessionTimeouts::getOfflineClientSessionMaxIdleMs : SessionTimeouts::getClientSessionMaxIdleMs;
if (idleTimeoutChecker.apply(sessionToImportInto.getRealm(), clientSession.getClient(), entity) == SessionTimeouts.ENTRY_EXPIRED_FLAG
|| lifespanChecker.apply(sessionToImportInto.getRealm(), clientSession.getClient(), entity) == SessionTimeouts.ENTRY_EXPIRED_FLAG) {
return null;
}
}
final UUID clientSessionId = entity.getId();
SessionUpdateTask<AuthenticatedClientSessionEntity> createClientSessionTask = Tasks.addIfAbsentSync();
@ -1072,10 +1102,12 @@ public class InfinispanUserSessionProvider implements UserSessionProvider {
}
private AuthenticatedClientSessionEntity createAuthenticatedClientSessionInstance(AuthenticatedClientSessionModel clientSession, String realmId, boolean offline) {
private AuthenticatedClientSessionEntity createAuthenticatedClientSessionInstance(AuthenticatedClientSessionModel clientSession,
String realmId, String clientId, boolean offline) {
final UUID clientSessionId = keyGenerator.generateKeyUUID(session, getClientSessionCache(offline));
AuthenticatedClientSessionEntity entity = new AuthenticatedClientSessionEntity(clientSessionId);
entity.setRealmId(realmId);
entity.setClientId(clientId);
entity.setAction(clientSession.getAction());
entity.setAuthMethod(clientSession.getProtocol());

View file

@ -27,6 +27,7 @@ import org.keycloak.common.Profile;
import org.keycloak.common.util.Environment;
import org.keycloak.common.util.Time;
import org.keycloak.connections.infinispan.InfinispanConnectionProvider;
import org.keycloak.models.ClientModel;
import org.keycloak.models.KeycloakSession;
import org.keycloak.models.KeycloakSessionFactory;
import org.keycloak.models.KeycloakSessionTask;
@ -66,7 +67,6 @@ import org.keycloak.provider.ProviderEventListener;
import java.io.Serializable;
import java.util.Set;
import java.util.UUID;
import java.util.function.BiFunction;
import static org.keycloak.models.sessions.infinispan.InfinispanAuthenticationSessionProviderFactory.PROVIDER_PRIORITY;
public class InfinispanUserSessionProviderFactory implements UserSessionProviderFactory, EnvironmentDependentProviderFactory {
@ -292,7 +292,7 @@ public class InfinispanUserSessionProviderFactory implements UserSessionProvider
}
private <K, V extends SessionEntity> RemoteCache checkRemoteCache(KeycloakSession session, Cache<K, SessionEntityWrapper<V>> ispnCache, RemoteCacheInvoker.MaxIdleTimeLoader maxIdleLoader,
BiFunction<RealmModel, V, Long> lifespanMsLoader, BiFunction<RealmModel, V, Long> maxIdleTimeMsLoader) {
SessionFunction<V> lifespanMsLoader, SessionFunction<V> maxIdleTimeMsLoader) {
Set<RemoteStore> remoteStores = InfinispanUtil.getRemoteStores(ispnCache);
if (remoteStores.isEmpty()) {

View file

@ -0,0 +1,38 @@
/*
* Copyright 2023 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;
import org.keycloak.models.ClientModel;
import org.keycloak.models.RealmModel;
import org.keycloak.models.sessions.infinispan.entities.SessionEntity;
/**
* <p>Function definition used for the lifespan and idle calculations for the infinispan
* session entities. The method receives the realm, client if needed (it's optional)
* and the entity. It returns the timestamp for the entity (lifespan, idle
* timeout,...) in milliseconds.</p>
*
* @param <V> The session entity to apply the function
*
* @author rmartinc
*/
@FunctionalInterface
public interface SessionFunction<V extends SessionEntity> {
Long apply(RealmModel realm, ClientModel client, V entity);
}

View file

@ -20,16 +20,17 @@ package org.keycloak.models.sessions.infinispan.changes;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.function.BiFunction;
import org.infinispan.Cache;
import org.infinispan.context.Flag;
import org.jboss.logging.Logger;
import org.keycloak.models.ClientModel;
import org.keycloak.models.AbstractKeycloakTransaction;
import org.keycloak.models.KeycloakSession;
import org.keycloak.models.RealmModel;
import org.keycloak.models.UserSessionModel;
import org.keycloak.models.sessions.infinispan.CacheDecorators;
import org.keycloak.models.sessions.infinispan.SessionFunction;
import org.keycloak.models.sessions.infinispan.entities.SessionEntity;
import org.keycloak.models.sessions.infinispan.remotestore.RemoteCacheInvoker;
import org.keycloak.connections.infinispan.InfinispanUtil;
@ -48,11 +49,11 @@ public class InfinispanChangelogBasedTransaction<K, V extends SessionEntity> ext
private final Map<K, SessionUpdatesList<V>> updates = new HashMap<>();
private final BiFunction<RealmModel, V, Long> lifespanMsLoader;
private final BiFunction<RealmModel, V, Long> maxIdleTimeMsLoader;
private final SessionFunction<V> lifespanMsLoader;
private final SessionFunction<V> maxIdleTimeMsLoader;
public InfinispanChangelogBasedTransaction(KeycloakSession kcSession, Cache<K, SessionEntityWrapper<V>> cache, RemoteCacheInvoker remoteCacheInvoker,
BiFunction<RealmModel, V, Long> lifespanMsLoader, BiFunction<RealmModel, V, Long> maxIdleTimeMsLoader) {
SessionFunction<V> lifespanMsLoader, SessionFunction<V> maxIdleTimeMsLoader) {
this.kcSession = kcSession;
this.cacheName = cache.getName();
this.cache = cache;
@ -162,8 +163,8 @@ public class InfinispanChangelogBasedTransaction<K, V extends SessionEntity> ext
RealmModel realm = sessionUpdates.getRealm();
long lifespanMs = lifespanMsLoader.apply(realm, sessionWrapper.getEntity());
long maxIdleTimeMs = maxIdleTimeMsLoader.apply(realm, sessionWrapper.getEntity());
long lifespanMs = lifespanMsLoader.apply(realm, sessionUpdates.getClient(), sessionWrapper.getEntity());
long maxIdleTimeMs = maxIdleTimeMsLoader.apply(realm, sessionUpdates.getClient(), sessionWrapper.getEntity());
MergedUpdate<V> merged = MergedUpdate.computeUpdate(sessionUpdates.getUpdateTasks(), sessionWrapper, lifespanMs, maxIdleTimeMs);

View file

@ -28,6 +28,9 @@ import java.util.concurrent.ConcurrentHashMap;
import org.infinispan.commons.marshall.Externalizer;
import org.infinispan.commons.marshall.MarshallUtil;
import org.infinispan.commons.marshall.SerializeWith;
import org.keycloak.models.ClientModel;
import org.keycloak.models.RealmModel;
import org.keycloak.models.sessions.infinispan.entities.AuthenticatedClientSessionEntity;
import org.keycloak.models.sessions.infinispan.entities.SessionEntity;
import java.util.HashMap;
import org.jboss.logging.Logger;
@ -96,6 +99,16 @@ public class SessionEntityWrapper<S extends SessionEntity> {
return entity;
}
public ClientModel getClientIfNeeded(RealmModel realm) {
if (entity instanceof AuthenticatedClientSessionEntity) {
String clientId = ((AuthenticatedClientSessionEntity) entity).getClientId();
if (clientId != null) {
return realm.getClientById(clientId);
}
}
return null;
}
public String getLocalMetadataNote(String key) {
if (isForTransport()) {
throw new IllegalStateException("This entity is only intended for transport");

View file

@ -20,6 +20,7 @@ package org.keycloak.models.sessions.infinispan.changes;
import java.util.LinkedList;
import java.util.List;
import org.keycloak.models.ClientModel;
import org.keycloak.models.RealmModel;
import org.keycloak.models.UserSessionModel;
import org.keycloak.models.sessions.infinispan.entities.SessionEntity;
@ -33,6 +34,8 @@ class SessionUpdatesList<S extends SessionEntity> {
private final RealmModel realm;
private final ClientModel client;
private final SessionEntityWrapper<S> entityWrapper;
private List<SessionUpdateTask<S>> updateTasks = new LinkedList<>();
@ -47,12 +50,17 @@ class SessionUpdatesList<S extends SessionEntity> {
this.realm = realm;
this.entityWrapper = entityWrapper;
this.persistenceState = persistenceState;
this.client = entityWrapper.getClientIfNeeded(realm);
}
public RealmModel getRealm() {
return realm;
}
public ClientModel getClient() {
return client;
}
public SessionEntityWrapper<S> getEntityWrapper() {
return entityWrapper;
}

View file

@ -27,6 +27,7 @@ import org.infinispan.commons.marshall.Externalizer;
import org.infinispan.commons.marshall.MarshallUtil;
import org.infinispan.commons.marshall.SerializeWith;
import org.jboss.logging.Logger;
import org.keycloak.models.AuthenticatedClientSessionModel;
import org.keycloak.models.sessions.infinispan.changes.SessionEntityWrapper;
import org.keycloak.models.sessions.infinispan.util.KeycloakMarshallUtil;
import java.util.UUID;
@ -42,6 +43,7 @@ public class AuthenticatedClientSessionEntity extends SessionEntity {
// Metadata attribute, which contains the last timestamp available on remoteCache. Used in decide whether we need to write to remoteCache (DC) or not
public static final String LAST_TIMESTAMP_REMOTE = "lstr";
public static final String CLIENT_ID_NOTE = "clientId";
private String authMethod;
private String redirectUri;
@ -83,6 +85,28 @@ public class AuthenticatedClientSessionEntity extends SessionEntity {
this.timestamp = timestamp;
}
public int getUserSessionStarted() {
String started = getNotes().get(AuthenticatedClientSessionModel.USER_SESSION_STARTED_AT_NOTE);
return started == null ? timestamp : Integer.parseInt(started);
}
public int getStarted() {
String started = getNotes().get(AuthenticatedClientSessionModel.STARTED_AT_NOTE);
return started == null ? timestamp : Integer.parseInt(started);
}
public boolean isUserSessionRememberMe() {
return Boolean.parseBoolean(getNotes().get(AuthenticatedClientSessionModel.USER_SESSION_REMEMBER_ME_NOTE));
}
public String getClientId() {
return getNotes().get(CLIENT_ID_NOTE);
}
public void setClientId(String clientId) {
getNotes().put(CLIENT_ID_NOTE, clientId);
}
public String getAction() {
return action;
}

View file

@ -31,20 +31,21 @@ import org.infinispan.context.Flag;
import org.jboss.logging.Logger;
import org.keycloak.connections.infinispan.TopologyInfo;
import org.keycloak.executors.ExecutorsProvider;
import org.keycloak.models.ClientModel;
import org.keycloak.models.KeycloakSession;
import org.keycloak.models.KeycloakSessionFactory;
import org.keycloak.models.RealmModel;
import org.keycloak.models.sessions.infinispan.SessionFunction;
import org.keycloak.models.sessions.infinispan.changes.SessionEntityWrapper;
import org.keycloak.models.sessions.infinispan.entities.SessionEntity;
import org.keycloak.models.utils.KeycloakModelUtils;
import org.keycloak.connections.infinispan.InfinispanUtil;
import java.util.Random;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.BiFunction;
import org.infinispan.client.hotrod.VersionedValue;
import org.keycloak.models.utils.KeycloakModelUtils;
/**
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
@ -60,8 +61,8 @@ public class RemoteCacheSessionListener<K, V extends SessionEntity> {
private RemoteCache<K, SessionEntityWrapper<V>> remoteCache;
private TopologyInfo topologyInfo;
private ClientListenerExecutorDecorator<K> executor;
private BiFunction<RealmModel, V, Long> lifespanMsLoader;
private BiFunction<RealmModel, V, Long> maxIdleTimeMsLoader;
private SessionFunction<V> lifespanMsLoader;
private SessionFunction<V> maxIdleTimeMsLoader;
private KeycloakSessionFactory sessionFactory;
@ -70,7 +71,7 @@ public class RemoteCacheSessionListener<K, V extends SessionEntity> {
protected void init(KeycloakSession session, Cache<K, SessionEntityWrapper<V>> cache, RemoteCache<K, SessionEntityWrapper<V>> remoteCache,
BiFunction<RealmModel, V, Long> lifespanMsLoader, BiFunction<RealmModel, V, Long> maxIdleTimeMsLoader) {
SessionFunction<V> lifespanMsLoader, SessionFunction<V> maxIdleTimeMsLoader) {
this.cache = cache;
this.remoteCache = remoteCache;
@ -135,8 +136,9 @@ public class RemoteCacheSessionListener<K, V extends SessionEntity> {
KeycloakModelUtils.runJobInTransaction(sessionFactory, (session -> {
RealmModel realm = session.realms().getRealm(newWrapper.getEntity().getRealmId());
long lifespanMs = lifespanMsLoader.apply(realm, newWrapper.getEntity());
long maxIdleTimeMs = maxIdleTimeMsLoader.apply(realm, newWrapper.getEntity());
ClientModel client = newWrapper.getClientIfNeeded(realm);
long lifespanMs = lifespanMsLoader.apply(realm, client, newWrapper.getEntity());
long maxIdleTimeMs = maxIdleTimeMsLoader.apply(realm, client, newWrapper.getEntity());
logger.tracef("Calling putIfAbsent for entity '%s' in the cache '%s' . lifespan: %d ms, maxIdleTime: %d ms", key, remoteCache.getName(), lifespanMs, maxIdleTimeMs);
@ -187,8 +189,9 @@ public class RemoteCacheSessionListener<K, V extends SessionEntity> {
KeycloakModelUtils.runJobInTransaction(sessionFactory, (session -> {
RealmModel realm = session.realms().getRealm(sessionWrapper.getEntity().getRealmId());
long lifespanMs = lifespanMsLoader.apply(realm, sessionWrapper.getEntity());
long maxIdleTimeMs = maxIdleTimeMsLoader.apply(realm, sessionWrapper.getEntity());
ClientModel client = sessionWrapper.getClientIfNeeded(realm);
long lifespanMs = lifespanMsLoader.apply(realm, client, sessionWrapper.getEntity());
long maxIdleTimeMs = maxIdleTimeMsLoader.apply(realm, client, sessionWrapper.getEntity());
// We received event from remoteCache, so we won't update it back
replaced.set(cache.getAdvancedCache().withFlags(Flag.SKIP_CACHE_STORE, Flag.SKIP_CACHE_LOAD, Flag.IGNORE_RETURN_VALUES)
@ -253,7 +256,7 @@ public class RemoteCacheSessionListener<K, V extends SessionEntity> {
public static <K, V extends SessionEntity> RemoteCacheSessionListener createListener(KeycloakSession session, Cache<K, SessionEntityWrapper<V>> cache, RemoteCache<K, SessionEntityWrapper<V>> remoteCache,
BiFunction<RealmModel, V, Long> lifespanMsLoader, BiFunction<RealmModel, V, Long> maxIdleTimeMsLoader) {
SessionFunction<V> lifespanMsLoader, SessionFunction<V> maxIdleTimeMsLoader) {
/*boolean isCoordinator = InfinispanUtil.isCoordinator(cache);
// Just cluster coordinator will fetch userSessions from remote cache.

View file

@ -18,12 +18,14 @@
package org.keycloak.models.sessions.infinispan.util;
import java.util.concurrent.TimeUnit;
import org.keycloak.common.util.Time;
import org.keycloak.models.ClientModel;
import org.keycloak.models.RealmModel;
import org.keycloak.models.sessions.infinispan.entities.AuthenticatedClientSessionEntity;
import org.keycloak.models.sessions.infinispan.entities.LoginFailureEntity;
import org.keycloak.models.sessions.infinispan.entities.UserSessionEntity;
import org.keycloak.models.utils.SessionTimeoutHelper;
import org.keycloak.models.utils.SessionExpirationUtils;
/**
* @author <a href="mailto:mposolda@redhat.com">Marek Posolda</a>
@ -45,52 +47,37 @@ public class SessionTimeouts {
* Returned value will be used as "lifespan" when calling put/replace operation in the infinispan cache for this entity
*
* @param realm
* @param client
* @param userSessionEntity
* @return
*/
public static long getUserSessionLifespanMs(RealmModel realm, UserSessionEntity userSessionEntity) {
int timeSinceSessionStart = Time.currentTime() - userSessionEntity.getStarted();
int sessionMaxLifespan = Math.max(realm.getSsoSessionMaxLifespan(), MINIMAL_EXPIRATION_SEC);
if (userSessionEntity.isRememberMe()) {
sessionMaxLifespan = Math.max(realm.getSsoSessionMaxLifespanRememberMe(), sessionMaxLifespan);
}
long timeToExpire = sessionMaxLifespan - timeSinceSessionStart;
// Indication that entry should be expired
if (timeToExpire <=0) {
public static long getUserSessionLifespanMs(RealmModel realm, ClientModel client, UserSessionEntity userSessionEntity) {
long lifespan = SessionExpirationUtils.calculateUserSessionMaxLifespanTimestamp(false, userSessionEntity.isRememberMe(),
TimeUnit.SECONDS.toMillis(userSessionEntity.getStarted()), realm);
lifespan = lifespan - Time.currentTimeMillis();
if (lifespan <= 0) {
return ENTRY_EXPIRED_FLAG;
}
return Time.toMillis(timeToExpire);
return lifespan;
}
/**
* Get the maximum idle time for this userSession.
* Returned value will be used when as "maxIdleTime" when calling put/replace operation in the infinispan cache for this entity
*
* @param realm
* @param client
* @param userSessionEntity
* @return
*/
public static long getUserSessionMaxIdleMs(RealmModel realm, UserSessionEntity userSessionEntity) {
int timeSinceLastRefresh = Time.currentTime() - userSessionEntity.getLastSessionRefresh();
int sessionIdleMs = Math.max(realm.getSsoSessionIdleTimeout(), MINIMAL_EXPIRATION_SEC);
if (userSessionEntity.isRememberMe()) {
sessionIdleMs = Math.max(realm.getSsoSessionIdleTimeoutRememberMe(), sessionIdleMs);
}
long maxIdleTime = sessionIdleMs - timeSinceLastRefresh + SessionTimeoutHelper.PERIODIC_CLEANER_IDLE_TIMEOUT_WINDOW_SECONDS;
// Indication that entry should be expired
if (maxIdleTime <=0) {
public static long getUserSessionMaxIdleMs(RealmModel realm, ClientModel client, UserSessionEntity userSessionEntity) {
long idle = SessionExpirationUtils.calculateUserSessionIdleTimestamp(false, userSessionEntity.isRememberMe(),
TimeUnit.SECONDS.toMillis(userSessionEntity.getLastSessionRefresh()), realm);
idle = idle - Time.currentTimeMillis();
if (idle <= 0) {
return ENTRY_EXPIRED_FLAG;
}
return Time.toMillis(maxIdleTime);
return idle;
}
@ -99,29 +86,19 @@ public class SessionTimeouts {
* Returned value will be used as "lifespan" when calling put/replace operation in the infinispan cache for this entity
*
* @param realm
* @param client
* @param clientSessionEntity
* @return
*/
public static long getClientSessionLifespanMs(RealmModel realm, AuthenticatedClientSessionEntity clientSessionEntity) {
int timeSinceTimestampUpdate = Time.currentTime() - clientSessionEntity.getTimestamp();
int sessionMaxLifespan = Math.max(realm.getSsoSessionMaxLifespan(), realm.getSsoSessionMaxLifespanRememberMe());
// clientSession max lifespan has preference if set
if (realm.getClientSessionMaxLifespan() > 0) {
sessionMaxLifespan = realm.getClientSessionMaxLifespan();
}
sessionMaxLifespan = Math.max(sessionMaxLifespan, MINIMAL_EXPIRATION_SEC);
long timeToExpire = sessionMaxLifespan - timeSinceTimestampUpdate;
// Indication that entry should be expired
if (timeToExpire <=0) {
public static long getClientSessionLifespanMs(RealmModel realm, ClientModel client, AuthenticatedClientSessionEntity clientSessionEntity) {
long lifespan = SessionExpirationUtils.calculateClientSessionMaxLifespanTimestamp(false, clientSessionEntity.isUserSessionRememberMe(),
TimeUnit.SECONDS.toMillis(clientSessionEntity.getStarted()), TimeUnit.SECONDS.toMillis(clientSessionEntity.getUserSessionStarted()),
realm, client);
lifespan = lifespan - Time.currentTimeMillis();
if (lifespan <= 0) {
return ENTRY_EXPIRED_FLAG;
}
return Time.toMillis(timeToExpire);
return lifespan;
}
@ -130,29 +107,18 @@ public class SessionTimeouts {
* Returned value will be used as "maxIdle" when calling put/replace operation in the infinispan cache for this entity
*
* @param realm
* @param client
* @param clientSessionEntity
* @return
*/
public static long getClientSessionMaxIdleMs(RealmModel realm, AuthenticatedClientSessionEntity clientSessionEntity) {
int timeSinceTimestampUpdate = Time.currentTime() - clientSessionEntity.getTimestamp();
int sessionIdleTimeout = Math.max(realm.getSsoSessionIdleTimeout(), realm.getSsoSessionIdleTimeoutRememberMe());
// clientSession idle timeout has preference if set
if (realm.getClientSessionIdleTimeout() > 0) {
sessionIdleTimeout = realm.getClientSessionIdleTimeout();
}
sessionIdleTimeout = Math.max(sessionIdleTimeout, MINIMAL_EXPIRATION_SEC);
long timeToExpire = sessionIdleTimeout - timeSinceTimestampUpdate + SessionTimeoutHelper.PERIODIC_CLEANER_IDLE_TIMEOUT_WINDOW_SECONDS;
// Indication that entry should be expired
if (timeToExpire <=0) {
public static long getClientSessionMaxIdleMs(RealmModel realm, ClientModel client, AuthenticatedClientSessionEntity clientSessionEntity) {
long idle = SessionExpirationUtils.calculateClientSessionIdleTimestamp(false, clientSessionEntity.isUserSessionRememberMe(),
TimeUnit.SECONDS.toMillis(clientSessionEntity.getTimestamp()), realm, client);
idle = idle - Time.currentTimeMillis();
if (idle <= 0) {
return ENTRY_EXPIRED_FLAG;
}
return Time.toMillis(timeToExpire);
return idle;
}
@ -161,25 +127,21 @@ public class SessionTimeouts {
* Returned value will be used as "lifespan" when calling put/replace operation in the infinispan cache for this entity
*
* @param realm
* @param client
* @param userSessionEntity
* @return
*/
public static long getOfflineSessionLifespanMs(RealmModel realm, UserSessionEntity userSessionEntity) {
// By default, this is disabled, so offlineSessions have just "maxIdle"
if (!realm.isOfflineSessionMaxLifespanEnabled()) return -1l;
int timeSinceSessionStart = Time.currentTime() - userSessionEntity.getStarted();
int sessionMaxLifespan = Math.max(realm.getOfflineSessionMaxLifespan(), MINIMAL_EXPIRATION_SEC);
long timeToExpire = sessionMaxLifespan - timeSinceSessionStart;
// Indication that entry should be expired
if (timeToExpire <=0) {
public static long getOfflineSessionLifespanMs(RealmModel realm, ClientModel client, UserSessionEntity userSessionEntity) {
long lifespan = SessionExpirationUtils.calculateUserSessionMaxLifespanTimestamp(true, userSessionEntity.isRememberMe(),
TimeUnit.SECONDS.toMillis(userSessionEntity.getStarted()), realm);
if (lifespan == -1L) {
return lifespan;
}
lifespan = lifespan - Time.currentTimeMillis();
if (lifespan <= 0) {
return ENTRY_EXPIRED_FLAG;
}
return Time.toMillis(timeToExpire);
return lifespan;
}
@ -188,22 +150,18 @@ public class SessionTimeouts {
* Returned value will be used when as "maxIdleTime" when calling put/replace operation in the infinispan cache for this entity
*
* @param realm
* @param client
* @param userSessionEntity
* @return
*/
public static long getOfflineSessionMaxIdleMs(RealmModel realm, UserSessionEntity userSessionEntity) {
int timeSinceLastRefresh = Time.currentTime() - userSessionEntity.getLastSessionRefresh();
int sessionIdle = Math.max(realm.getOfflineSessionIdleTimeout(), MINIMAL_EXPIRATION_SEC);
long maxIdleTime = sessionIdle - timeSinceLastRefresh + SessionTimeoutHelper.PERIODIC_CLEANER_IDLE_TIMEOUT_WINDOW_SECONDS;
// Indication that entry should be expired
if (maxIdleTime <=0) {
public static long getOfflineSessionMaxIdleMs(RealmModel realm, ClientModel client, UserSessionEntity userSessionEntity) {
long idle = SessionExpirationUtils.calculateUserSessionIdleTimestamp(true, userSessionEntity.isRememberMe(),
TimeUnit.SECONDS.toMillis(userSessionEntity.getLastSessionRefresh()), realm);
idle = idle - Time.currentTimeMillis();
if (idle <= 0) {
return ENTRY_EXPIRED_FLAG;
}
return Time.toMillis(maxIdleTime);
return idle;
}
/**
@ -211,30 +169,22 @@ public class SessionTimeouts {
* Returned value will be used as "lifespan" when calling put/replace operation in the infinispan cache for this entity
*
* @param realm
* @param client
* @param authenticatedClientSessionEntity
* @return
*/
public static long getOfflineClientSessionLifespanMs(RealmModel realm, AuthenticatedClientSessionEntity authenticatedClientSessionEntity) {
// By default, this is disabled, so offlineSessions have just "maxIdle"
if (!realm.isOfflineSessionMaxLifespanEnabled() && realm.getClientOfflineSessionMaxLifespan() <= 0) return -1l;
int timeSinceTimestamp = Time.currentTime() - authenticatedClientSessionEntity.getTimestamp();
int sessionMaxLifespan = Math.max(realm.getOfflineSessionMaxLifespan(), MINIMAL_EXPIRATION_SEC);
// clientSession max lifespan has preference if set
if (realm.getClientOfflineSessionMaxLifespan() > 0) {
sessionMaxLifespan = realm.getClientOfflineSessionMaxLifespan();
public static long getOfflineClientSessionLifespanMs(RealmModel realm, ClientModel client, AuthenticatedClientSessionEntity authenticatedClientSessionEntity) {
long lifespan = SessionExpirationUtils.calculateClientSessionMaxLifespanTimestamp(true, authenticatedClientSessionEntity.isUserSessionRememberMe(),
TimeUnit.SECONDS.toMillis(authenticatedClientSessionEntity.getStarted()), TimeUnit.SECONDS.toMillis(authenticatedClientSessionEntity.getUserSessionStarted()),
realm, client);
if (lifespan == -1L) {
return lifespan;
}
long timeToExpire = sessionMaxLifespan - timeSinceTimestamp;
// Indication that entry should be expired
if (timeToExpire <=0) {
lifespan = lifespan - Time.currentTimeMillis();
if (lifespan <= 0) {
return ENTRY_EXPIRED_FLAG;
}
return Time.toMillis(timeToExpire);
return lifespan;
}
/**
@ -242,27 +192,18 @@ public class SessionTimeouts {
* Returned value will be used as "maxIdle" when calling put/replace operation in the infinispan cache for this entity
*
* @param realm
* @param client
* @param authenticatedClientSessionEntity
* @return
*/
public static long getOfflineClientSessionMaxIdleMs(RealmModel realm, AuthenticatedClientSessionEntity authenticatedClientSessionEntity) {
int timeSinceLastRefresh = Time.currentTime() - authenticatedClientSessionEntity.getTimestamp();
int sessionIdle = Math.max(realm.getOfflineSessionIdleTimeout(), MINIMAL_EXPIRATION_SEC);
// clientSession idle timeout has preference if set
if (realm.getClientOfflineSessionIdleTimeout() > 0) {
sessionIdle = realm.getClientOfflineSessionIdleTimeout();
}
long maxIdleTime = sessionIdle - timeSinceLastRefresh + SessionTimeoutHelper.PERIODIC_CLEANER_IDLE_TIMEOUT_WINDOW_SECONDS;
// Indication that entry should be expired
if (maxIdleTime <=0) {
public static long getOfflineClientSessionMaxIdleMs(RealmModel realm, ClientModel client, AuthenticatedClientSessionEntity authenticatedClientSessionEntity) {
long idle = SessionExpirationUtils.calculateClientSessionIdleTimestamp(true, authenticatedClientSessionEntity.isUserSessionRememberMe(),
TimeUnit.SECONDS.toMillis(authenticatedClientSessionEntity.getTimestamp()), realm, client);
idle = idle - Time.currentTimeMillis();
if (idle <= 0) {
return ENTRY_EXPIRED_FLAG;
}
return Time.toMillis(maxIdleTime);
return idle;
}
@ -270,10 +211,11 @@ public class SessionTimeouts {
* Not using lifespan for detached login failure (backwards compatibility with the background cleaner threads, which were used for cleanup of detached login failures)
*
* @param realm
* @param client
* @param loginFailureEntity
* @return
*/
public static long getLoginFailuresLifespanMs(RealmModel realm, LoginFailureEntity loginFailureEntity) {
public static long getLoginFailuresLifespanMs(RealmModel realm, ClientModel client, LoginFailureEntity loginFailureEntity) {
return -1l;
}
@ -282,12 +224,11 @@ public class SessionTimeouts {
* Not using maxIdle for detached login failure (backwards compatibility with the background cleaner threads, which were used for cleanup of detached login failures)
*
* @param realm
* @param client
* @param loginFailureEntity
* @return
*/
public static long getLoginFailuresMaxIdleMs(RealmModel realm, LoginFailureEntity loginFailureEntity) {
public static long getLoginFailuresMaxIdleMs(RealmModel realm, ClientModel client, LoginFailureEntity loginFailureEntity) {
return -1l;
}
}

View file

@ -234,7 +234,9 @@ public class HotRodCrudOperations<K, E extends AbstractHotRodEntity, V extends A
// workaround if the query contains us.clientId field, in which case don't read by id => read without optimistic locking.
// See https://issues.redhat.com/browse/ISPN-14537
if (!dmc.isEmpty() && dmc.partiallyEvaluate((field, op, arg) -> field == UserSessionModel.SearchableFields.CLIENT_ID).toString().contains("__TRUE__")) {
if (!dmc.isEmpty() && dmc.partiallyEvaluate((field, op, arg) ->
field == UserSessionModel.SearchableFields.CLIENT_ID || field == UserSessionModel.SearchableFields.CORRESPONDING_SESSION_ID
).toString().contains("__TRUE__")) {
Query<E> query = prepareQueryWithPrefixAndParameters(null, queryParameters);
CloseableIterator<E> iterator = paginateQuery(query, queryParameters.getOffset(),
queryParameters.getLimit()).iterator();

View file

@ -124,11 +124,14 @@ public class MapUserSessionProvider implements UserSessionProvider {
}
MapAuthenticatedClientSessionEntity entity = createAuthenticatedClientSessionEntityInstance(null, userSession.getId(),
realm.getId(), client.getId(), false);
realm.getId(), client.getId(), userSession.isOffline());
String started = entity.getTimestamp() != null ? String.valueOf(TimeAdapter.fromMilliSecondsToSeconds(entity.getTimestamp())) : String.valueOf(0);
entity.setNote(AuthenticatedClientSessionModel.STARTED_AT_NOTE, started);
entity.setNote(AuthenticatedClientSessionModel.USER_SESSION_STARTED_AT_NOTE, String.valueOf(userSession.getStarted()));
if (userSession.isRememberMe()) {
entity.setNote(AuthenticatedClientSessionModel.USER_SESSION_REMEMBER_ME_NOTE, "true");
}
setClientSessionExpiration(entity, realm, client);
userSessionEntity.addAuthenticatedClientSession(entity);
// We need to load the clientSession through userModel so we return an entity that is included within the
@ -427,6 +430,7 @@ public class MapUserSessionProvider implements UserSessionProvider {
MapAuthenticatedClientSessionEntity clientSessionEntity = createAuthenticatedClientSessionInstance(clientSession, offlineUserSession, true);
int currentTime = Time.currentTime();
clientSessionEntity.setNote(AuthenticatedClientSessionModel.STARTED_AT_NOTE, String.valueOf(currentTime));
clientSessionEntity.setNote(AuthenticatedClientSessionModel.USER_SESSION_STARTED_AT_NOTE, String.valueOf(offlineUserSession.getStarted()));
clientSessionEntity.setTimestamp(Time.currentTimeMillis());
RealmModel realm = clientSession.getRealm();
setClientSessionExpiration(clientSessionEntity, realm, clientSession.getClient());

View file

@ -17,9 +17,11 @@
package org.keycloak.models.map.userSession;
import org.keycloak.common.util.Time;
import org.keycloak.models.AuthenticatedClientSessionModel;
import org.keycloak.models.ClientModel;
import org.keycloak.models.RealmModel;
import org.keycloak.models.map.common.TimeAdapter;
import org.keycloak.models.utils.SessionExpirationUtils;
import org.keycloak.protocol.oidc.OIDCConfigAttributes;
/**
@ -27,131 +29,46 @@ import org.keycloak.protocol.oidc.OIDCConfigAttributes;
*/
public class SessionExpiration {
private static long getTimestampNote(MapAuthenticatedClientSessionEntity entity, String name) {
String value = entity.getNote(name);
if (value == null) {
// return timestamp if not found
return entity.getTimestamp();
}
return TimeAdapter.fromSecondsToMilliseconds(Integer.parseInt(value));
}
public static void setClientSessionExpiration(MapAuthenticatedClientSessionEntity entity, RealmModel realm, ClientModel client) {
long timestampMillis = entity.getTimestamp() != null ? entity.getTimestamp() : 0L;
if (Boolean.TRUE.equals(entity.isOffline())) {
long sessionExpires = timestampMillis + TimeAdapter.fromSecondsToMilliseconds(realm.getOfflineSessionIdleTimeout());
if (realm.isOfflineSessionMaxLifespanEnabled()) {
sessionExpires = timestampMillis + TimeAdapter.fromSecondsToMilliseconds(realm.getOfflineSessionMaxLifespan());
long clientSessionStartedAtMillis = getTimestampNote(entity, AuthenticatedClientSessionModel.STARTED_AT_NOTE);
long userSessionStartedAtMillis = getTimestampNote(entity, AuthenticatedClientSessionModel.USER_SESSION_STARTED_AT_NOTE);
boolean isRememberMe = Boolean.parseBoolean(entity.getNote(AuthenticatedClientSessionModel.USER_SESSION_REMEMBER_ME_NOTE));
boolean isOffline = Boolean.TRUE.equals(entity.isOffline());
long clientOfflineSessionMaxLifespan;
String clientOfflineSessionMaxLifespanPerClient = client.getAttribute(OIDCConfigAttributes.CLIENT_OFFLINE_SESSION_MAX_LIFESPAN);
if (clientOfflineSessionMaxLifespanPerClient != null && !clientOfflineSessionMaxLifespanPerClient.trim().isEmpty()) {
clientOfflineSessionMaxLifespan = TimeAdapter.fromSecondsToMilliseconds(Long.parseLong(clientOfflineSessionMaxLifespanPerClient));
long expiresbyLifespan = SessionExpirationUtils.calculateClientSessionMaxLifespanTimestamp(isOffline, isRememberMe,
clientSessionStartedAtMillis, userSessionStartedAtMillis, realm, client);
long expiresByIdle =SessionExpirationUtils.calculateClientSessionIdleTimestamp(isOffline, isRememberMe, timestampMillis, realm, client);
if (expiresbyLifespan > 0) {
entity.setExpiration(Math.min(expiresbyLifespan, expiresByIdle));
} else {
clientOfflineSessionMaxLifespan = TimeAdapter.fromSecondsToMilliseconds(realm.getClientOfflineSessionMaxLifespan());
}
if (clientOfflineSessionMaxLifespan > 0) {
long clientOfflineSessionMaxExpiration = timestampMillis + clientOfflineSessionMaxLifespan;
sessionExpires = Math.min(sessionExpires, clientOfflineSessionMaxExpiration);
}
}
long expiration = timestampMillis + TimeAdapter.fromSecondsToMilliseconds(realm.getOfflineSessionIdleTimeout());
long clientOfflineSessionIdleTimeout;
String clientOfflineSessionIdleTimeoutPerClient = client.getAttribute(OIDCConfigAttributes.CLIENT_OFFLINE_SESSION_IDLE_TIMEOUT);
if (clientOfflineSessionIdleTimeoutPerClient != null && !clientOfflineSessionIdleTimeoutPerClient.trim().isEmpty()) {
clientOfflineSessionIdleTimeout = TimeAdapter.fromSecondsToMilliseconds(Long.parseLong(clientOfflineSessionIdleTimeoutPerClient));
} else {
clientOfflineSessionIdleTimeout = TimeAdapter.fromSecondsToMilliseconds(realm.getClientOfflineSessionIdleTimeout());
}
if (clientOfflineSessionIdleTimeout > 0) {
long clientOfflineSessionIdleExpiration = timestampMillis + clientOfflineSessionIdleTimeout;
expiration = Math.min(expiration, clientOfflineSessionIdleExpiration);
}
entity.setExpiration(Math.min(expiration, sessionExpires));
} else {
long sessionExpires = timestampMillis + (realm.getSsoSessionMaxLifespanRememberMe() > 0
? TimeAdapter.fromSecondsToMilliseconds(realm.getSsoSessionMaxLifespanRememberMe()) : TimeAdapter.fromSecondsToMilliseconds(realm.getSsoSessionMaxLifespan()));
long clientSessionMaxLifespan;
String clientSessionMaxLifespanPerClient = client.getAttribute(OIDCConfigAttributes.CLIENT_SESSION_MAX_LIFESPAN);
if (clientSessionMaxLifespanPerClient != null && !clientSessionMaxLifespanPerClient.trim().isEmpty()) {
clientSessionMaxLifespan = TimeAdapter.fromSecondsToMilliseconds(Long.parseLong(clientSessionMaxLifespanPerClient));
} else {
clientSessionMaxLifespan = TimeAdapter.fromSecondsToMilliseconds(realm.getClientSessionMaxLifespan());
}
if (clientSessionMaxLifespan > 0) {
long clientSessionMaxExpiration = timestampMillis + clientSessionMaxLifespan;
sessionExpires = Math.min(sessionExpires, clientSessionMaxExpiration);
}
long expiration = timestampMillis + (realm.getSsoSessionIdleTimeoutRememberMe() > 0
? TimeAdapter.fromSecondsToMilliseconds(realm.getSsoSessionIdleTimeoutRememberMe()) : TimeAdapter.fromSecondsToMilliseconds(realm.getSsoSessionIdleTimeout()));
long clientSessionIdleTimeout;
String clientSessionIdleTimeoutPerClient = client.getAttribute(OIDCConfigAttributes.CLIENT_SESSION_IDLE_TIMEOUT);
if (clientSessionIdleTimeoutPerClient != null && !clientSessionIdleTimeoutPerClient.trim().isEmpty()) {
clientSessionIdleTimeout = TimeAdapter.fromSecondsToMilliseconds(Long.parseLong(clientSessionIdleTimeoutPerClient));
} else {
clientSessionIdleTimeout = TimeAdapter.fromSecondsToMilliseconds(realm.getClientSessionIdleTimeout());
}
if (clientSessionIdleTimeout > 0) {
long clientSessionIdleExpiration = timestampMillis + clientSessionIdleTimeout;
expiration = Math.min(expiration, clientSessionIdleExpiration);
}
entity.setExpiration(Math.min(expiration, sessionExpires));
entity.setExpiration(expiresByIdle);
}
}
public static void setUserSessionExpiration(MapUserSessionEntity entity, RealmModel realm) {
long timestampMillis = entity.getTimestamp() != null ? entity.getTimestamp() : 0L;
long lastSessionRefreshMillis = entity.getLastSessionRefresh() != null ? entity.getLastSessionRefresh() : 0L;
if (Boolean.TRUE.equals(entity.isOffline())) {
long sessionExpires = lastSessionRefreshMillis + TimeAdapter.fromSecondsToMilliseconds(realm.getOfflineSessionIdleTimeout());
if (realm.isOfflineSessionMaxLifespanEnabled()) {
sessionExpires = timestampMillis + TimeAdapter.fromSecondsToMilliseconds(realm.getOfflineSessionMaxLifespan());
boolean isRememberMe = Boolean.TRUE.equals(entity.isRememberMe());
boolean isOffline = Boolean.TRUE.equals(entity.isOffline());
long clientOfflineSessionMaxLifespan = TimeAdapter.fromSecondsToMilliseconds(realm.getClientOfflineSessionMaxLifespan());
long expiresByLifespan = SessionExpirationUtils.calculateUserSessionMaxLifespanTimestamp(isOffline, isRememberMe, timestampMillis, realm);
long expiresByIdle = SessionExpirationUtils.calculateUserSessionIdleTimestamp(isOffline, isRememberMe, lastSessionRefreshMillis, realm);
if (clientOfflineSessionMaxLifespan > 0) {
long clientOfflineSessionMaxExpiration = timestampMillis + clientOfflineSessionMaxLifespan;
sessionExpires = Math.min(sessionExpires, clientOfflineSessionMaxExpiration);
}
}
long expiration = lastSessionRefreshMillis + TimeAdapter.fromSecondsToMilliseconds(realm.getOfflineSessionIdleTimeout());
long clientOfflineSessionIdleTimeout = TimeAdapter.fromSecondsToMilliseconds(realm.getClientOfflineSessionIdleTimeout());
if (clientOfflineSessionIdleTimeout > 0) {
long clientOfflineSessionIdleExpiration = Time.currentTimeMillis() + clientOfflineSessionIdleTimeout;
expiration = Math.min(expiration, clientOfflineSessionIdleExpiration);
}
entity.setExpiration(Math.min(expiration, sessionExpires));
if (expiresByLifespan > 0) {
entity.setExpiration(Math.min(expiresByLifespan, expiresByIdle));
} else {
long sessionExpires = timestampMillis
+ (Boolean.TRUE.equals(entity.isRememberMe()) && realm.getSsoSessionMaxLifespanRememberMe() > 0
? TimeAdapter.fromSecondsToMilliseconds(realm.getSsoSessionMaxLifespanRememberMe())
: TimeAdapter.fromSecondsToMilliseconds(realm.getSsoSessionMaxLifespan()));
long clientSessionMaxLifespan = TimeAdapter.fromSecondsToMilliseconds(realm.getClientSessionMaxLifespan());
if (clientSessionMaxLifespan > 0) {
long clientSessionMaxExpiration = timestampMillis + clientSessionMaxLifespan;
sessionExpires = Math.min(sessionExpires, clientSessionMaxExpiration);
}
long expiration = lastSessionRefreshMillis + (Boolean.TRUE.equals(entity.isRememberMe()) && realm.getSsoSessionIdleTimeoutRememberMe() > 0
? TimeAdapter.fromSecondsToMilliseconds(realm.getSsoSessionIdleTimeoutRememberMe())
: TimeAdapter.fromSecondsToMilliseconds(realm.getSsoSessionIdleTimeout()));
long clientSessionIdleTimeout = TimeAdapter.fromSecondsToMilliseconds(realm.getClientSessionIdleTimeout());
if (clientSessionIdleTimeout > 0) {
long clientSessionIdleExpiration = lastSessionRefreshMillis + clientSessionIdleTimeout;
expiration = Math.min(expiration, clientSessionIdleExpiration);
}
entity.setExpiration(Math.min(expiration, sessionExpires));
entity.setExpiration(expiresByIdle);
}
}
}

View file

@ -64,6 +64,9 @@ public final class Constants {
public static final int DEFAULT_OFFLINE_SESSION_MAX_LIFESPAN = 5184000;
public static final String DEFAULT_SIGNATURE_ALGORITHM = Algorithm.RS256;
public static final int DEFAULT_SESSION_IDLE_TIMEOUT = 1800; // 30 minutes
public static final int DEFAULT_SESSION_MAX_LIFESPAN = 36000; // 10 hours
public static final String DEFAULT_WEBAUTHN_POLICY_SIGNATURE_ALGORITHMS = Algorithm.ES256;
public static final String DEFAULT_WEBAUTHN_POLICY_RP_ENTITY_NAME = "keycloak";
// it stands for optional parameter not specified in WebAuthn

View file

@ -0,0 +1,203 @@
/*
* Copyright 2023 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.utils;
import java.util.concurrent.TimeUnit;
import org.keycloak.models.ClientModel;
import org.keycloak.models.Constants;
import org.keycloak.models.RealmModel;
import org.keycloak.protocol.oidc.OIDCConfigAttributes;
/**
* <p>Shared methods to calculate the session expiration and idle.</p>
*
* @author rmartinc
*/
public class SessionExpirationUtils {
/**
* Calculates the time in which the session is expired via max lifetime
* configuration.
* @param offline is the session offline?
* @param isRememberMe is the session remember me?
* @param created timestamp when the session was created
* @param realm The realm model
* @return The time when the user session is expired or -1 if does not expire
*/
public static long calculateUserSessionMaxLifespanTimestamp(boolean offline, boolean isRememberMe, long created, RealmModel realm) {
long timestamp = -1;
if (offline) {
if (realm.isOfflineSessionMaxLifespanEnabled()) {
timestamp = created + TimeUnit.SECONDS.toMillis(getOfflineSessionMaxLifespan(realm));
}
} else {
long userSessionMaxLifespan = TimeUnit.SECONDS.toMillis(getSsoSessionMaxLifespan(realm));
if (isRememberMe) {
userSessionMaxLifespan = Math.max(userSessionMaxLifespan, TimeUnit.SECONDS.toMillis(realm.getSsoSessionMaxLifespanRememberMe()));
}
timestamp = created + userSessionMaxLifespan;
}
return timestamp;
}
/**
* Calculates the time in which the user session is expired via the idle
* configuration.
* @param offline is the session offline?
* @param isRememberMe is the session remember me?
* @param lastRefreshed The last time the session was refreshed
* @param realm The realm model
* @return The time in which the user session is expired by idle timeout
*/
public static long calculateUserSessionIdleTimestamp(boolean offline, boolean isRememberMe, long lastRefreshed, RealmModel realm) {
long timestamp;
if (offline) {
timestamp = lastRefreshed + TimeUnit.SECONDS.toMillis(getOfflineSessionIdleTimeout(realm));
} else {
long userSessionIdleTimeout = TimeUnit.SECONDS.toMillis(getSsoSessionIdleTimeout(realm));
if (isRememberMe) {
userSessionIdleTimeout = Math.max(userSessionIdleTimeout, TimeUnit.SECONDS.toMillis(realm.getSsoSessionIdleTimeoutRememberMe()));
}
timestamp = lastRefreshed + userSessionIdleTimeout;
}
return timestamp;
}
/**
* Calculates the time in which the client session is expired via lifespan
* configuration in the realm and client.
* @param offline is the session offline?
* @param isRememberMe is the session remember me?
* @param clientSessionCreated timestamp when the client session was created
* @param userSessionCreated timestamp when the user session was created
* @param realm The realm model
* @param client The client model
* @return The time when the client session is expired or -1 if does not expire
*/
public static long calculateClientSessionMaxLifespanTimestamp(boolean offline, boolean isRememberMe,
long clientSessionCreated, long userSessionCreated, RealmModel realm, ClientModel client) {
long timestamp = -1;
if (offline) {
if (realm.isOfflineSessionMaxLifespanEnabled()) {
long clientOfflineSessionMaxLifespan = TimeUnit.SECONDS.toMillis(getOfflineSessionMaxLifespan(realm));
String clientOfflineSessionMaxLifespanPerClient = client == null? null : client.getAttribute(OIDCConfigAttributes.CLIENT_OFFLINE_SESSION_MAX_LIFESPAN);
if (clientOfflineSessionMaxLifespanPerClient != null && !clientOfflineSessionMaxLifespanPerClient.trim().isEmpty()) {
clientOfflineSessionMaxLifespan = TimeUnit.SECONDS.toMillis(Long.parseLong(clientOfflineSessionMaxLifespanPerClient));
} else if (realm.getClientOfflineSessionMaxLifespan() > 0) {
clientOfflineSessionMaxLifespan = TimeUnit.SECONDS.toMillis(realm.getClientOfflineSessionMaxLifespan());
}
timestamp = clientSessionCreated + clientOfflineSessionMaxLifespan;
long userSessionExpires = calculateUserSessionMaxLifespanTimestamp(offline, isRememberMe, userSessionCreated, realm);
timestamp = Math.min(timestamp, userSessionExpires);
}
} else {
long clientSessionMaxLifespan = TimeUnit.SECONDS.toMillis(getSsoSessionMaxLifespan(realm));
if (isRememberMe) {
clientSessionMaxLifespan = Math.max(clientSessionMaxLifespan, TimeUnit.SECONDS.toMillis(realm.getSsoSessionMaxLifespanRememberMe()));
}
String clientSessionMaxLifespanPerClient = client == null? null : client.getAttribute(OIDCConfigAttributes.CLIENT_SESSION_MAX_LIFESPAN);
if (clientSessionMaxLifespanPerClient != null && !clientSessionMaxLifespanPerClient.trim().isEmpty()) {
clientSessionMaxLifespan = TimeUnit.SECONDS.toMillis(Long.parseLong(clientSessionMaxLifespanPerClient));
} else if (realm.getClientSessionMaxLifespan() > 0) {
clientSessionMaxLifespan = TimeUnit.SECONDS.toMillis(realm.getClientSessionMaxLifespan());
}
timestamp = clientSessionCreated + clientSessionMaxLifespan;
long userSessionExpires = calculateUserSessionMaxLifespanTimestamp(offline, isRememberMe, userSessionCreated, realm);
timestamp = Math.min(timestamp, userSessionExpires);
}
return timestamp;
}
/**
* Calculates the time in which the user session is expired via the idle
* configuration in the realm and client.
* @param offline is the session offline?
* @param isRememberMe is the session remember me?
* @param lastRefreshed the last time the client session was refreshed
* @param realm the realm model
* @param client the client model
* @return The time in which the client session is expired by idle timeout
*/
public static long calculateClientSessionIdleTimestamp(boolean offline, boolean isRememberMe, long lastRefreshed,
RealmModel realm, ClientModel client) {
long timestamp;
if (offline) {
long clientOfflineSessionIdleTimeout = TimeUnit.SECONDS.toMillis(getOfflineSessionIdleTimeout(realm));
String clientOfflineSessionIdleTimeoutPerClient = client == null? null : client.getAttribute(OIDCConfigAttributes.CLIENT_OFFLINE_SESSION_IDLE_TIMEOUT);
if (clientOfflineSessionIdleTimeoutPerClient != null && !clientOfflineSessionIdleTimeoutPerClient.trim().isEmpty()) {
clientOfflineSessionIdleTimeout = TimeUnit.SECONDS.toMillis(Long.parseLong(clientOfflineSessionIdleTimeoutPerClient));
} else if (realm.getClientOfflineSessionIdleTimeout() > 0) {
clientOfflineSessionIdleTimeout = TimeUnit.SECONDS.toMillis(realm.getClientOfflineSessionIdleTimeout());
}
timestamp = lastRefreshed + clientOfflineSessionIdleTimeout;
} else {
long clientSessionIdleTimeout = TimeUnit.SECONDS.toMillis(getSsoSessionIdleTimeout(realm));
if (isRememberMe) {
clientSessionIdleTimeout = Math.max(clientSessionIdleTimeout, TimeUnit.SECONDS.toMillis(realm.getSsoSessionIdleTimeoutRememberMe()));
}
String clientSessionIdleTimeoutPerClient = client == null? null : client.getAttribute(OIDCConfigAttributes.CLIENT_SESSION_IDLE_TIMEOUT);
if (clientSessionIdleTimeoutPerClient != null && !clientSessionIdleTimeoutPerClient.trim().isEmpty()) {
clientSessionIdleTimeout = TimeUnit.SECONDS.toMillis(Long.parseLong(clientSessionIdleTimeoutPerClient));
} else if (realm.getClientSessionIdleTimeout() > 0){
clientSessionIdleTimeout = TimeUnit.SECONDS.toMillis(realm.getClientSessionIdleTimeout());
}
timestamp = lastRefreshed + clientSessionIdleTimeout;
}
return timestamp;
}
private static int getSsoSessionMaxLifespan(RealmModel realm) {
int lifespan = realm.getSsoSessionMaxLifespan();
if (lifespan <= 0) {
lifespan = Constants.DEFAULT_SESSION_MAX_LIFESPAN;
}
return lifespan;
}
private static int getOfflineSessionMaxLifespan(RealmModel realm) {
int lifespan = realm.getOfflineSessionMaxLifespan();
if (lifespan <= 0) {
lifespan = Constants.DEFAULT_OFFLINE_SESSION_MAX_LIFESPAN;
}
return lifespan;
}
private static int getSsoSessionIdleTimeout(RealmModel realm) {
int idle = realm.getSsoSessionIdleTimeout();
if (idle <= 0) {
idle = Constants.DEFAULT_SESSION_IDLE_TIMEOUT;
}
return idle;
}
private static int getOfflineSessionIdleTimeout(RealmModel realm) {
int idle = realm.getOfflineSessionIdleTimeout();
if (idle <= 0) {
idle = Constants.DEFAULT_OFFLINE_SESSION_IDLE_TIMEOUT;
}
return idle;
}
}

View file

@ -0,0 +1,257 @@
/*
* Copyright 2023 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.utils;
import java.lang.reflect.Proxy;
import java.util.HashMap;
import java.util.Map;
import org.junit.Assert;
import org.junit.Test;
import org.keycloak.common.util.Time;
import org.keycloak.models.ClientModel;
import org.keycloak.models.Constants;
import org.keycloak.models.RealmModel;
import org.keycloak.protocol.oidc.OIDCConfigAttributes;
/**
* <p>Class to perform unit tests of the SessionExpirationUtils class.</p>
*
* @author rmartinc
*/
public class SessionExpirationUtilsTest {
private static final Map<String, Object> realmMap = new HashMap<>();
private static final Map<String, Object> clientMap = new HashMap<>();
private static final RealmModel realm = createRealm();
private static final ClientModel client = createClient();
private static RealmModel createRealm() {
RealmModel realmModel = (RealmModel) Proxy.newProxyInstance(SessionExpirationUtilsTest.class.getClassLoader(),
new Class[]{RealmModel.class}, (proxy, method, args) -> {
Object result = realmMap.get(method.getName());
if (result != null) {
return result;
}
throw new UnsupportedOperationException("Realm method not in map: " + method.getName());
});
return realmModel;
}
private static ClientModel createClient() {
ClientModel clientModel = (ClientModel) Proxy.newProxyInstance(SessionExpirationUtilsTest.class.getClassLoader(),
new Class[]{ClientModel.class}, (proxy, method, args) -> {
if ("getAttribute".equals(method.getName()) && args.length == 1) {
return clientMap.get((String) args[0]);
}
throw new UnsupportedOperationException("Client method not in map: " + method.getName());
});
return clientModel;
}
private static void resetRealm() {
realmMap.put("isOfflineSessionMaxLifespanEnabled", false);
realmMap.put("getOfflineSessionMaxLifespan", 0);
realmMap.put("getOfflineSessionIdleTimeout", 0);
realmMap.put("getClientOfflineSessionMaxLifespan", 0);
realmMap.put("getClientOfflineSessionIdleTimeout", 0);
realmMap.put("getSsoSessionMaxLifespan", 0);
realmMap.put("getSsoSessionIdleTimeout", 0);
realmMap.put("getClientSessionMaxLifespan", 0);
realmMap.put("getClientSessionIdleTimeout", 0);
realmMap.put("getSsoSessionMaxLifespanRememberMe", 0);
realmMap.put("getSsoSessionIdleTimeoutRememberMe", 0);
}
private static void resetClient() {
clientMap.clear();
}
@Test
public void testCalculateUserSessionMaxLifespanTimestampOnline() {
long t = Time.currentTimeMillis();
resetRealm();
// non valid lifespan 0 or negative is default 36000
Assert.assertEquals(Constants.DEFAULT_SESSION_MAX_LIFESPAN * 1000L,
SessionExpirationUtils.calculateUserSessionMaxLifespanTimestamp(false, false, t, realm) - t);
// normal lifespan to 1000s
realmMap.put("getSsoSessionMaxLifespan", 1000);
Assert.assertEquals(1000 * 1000L, SessionExpirationUtils.calculateUserSessionMaxLifespanTimestamp(false, false, t, realm) - t);
// use remember me
realmMap.put("getSsoSessionMaxLifespanRememberMe", 2000);
Assert.assertEquals(2000 * 1000L, SessionExpirationUtils.calculateUserSessionMaxLifespanTimestamp(false, true, t, realm) - t);
}
@Test
public void testCalculateUserSessionMaxLifespanTimestampOffline() {
long t = Time.currentTimeMillis();
resetRealm();
// not activated expiration for offline
Assert.assertEquals(-1L, SessionExpirationUtils.calculateUserSessionMaxLifespanTimestamp(true, false, t, realm));
// activate and 0 should be default
realmMap.put("isOfflineSessionMaxLifespanEnabled", true);
Assert.assertEquals(Constants.DEFAULT_OFFLINE_SESSION_MAX_LIFESPAN * 1000L,
SessionExpirationUtils.calculateUserSessionMaxLifespanTimestamp(true, false, t, realm) - t);
// normal lifespan 2000
realmMap.put("getOfflineSessionMaxLifespan", 2000);
Assert.assertEquals(2000 * 1000L,
SessionExpirationUtils.calculateUserSessionMaxLifespanTimestamp(true, false, t, realm) - t);
// remember me does not affect offline
realmMap.put("getSsoSessionMaxLifespanRememberMe", 4000);
Assert.assertEquals(2000 * 1000L,
SessionExpirationUtils.calculateUserSessionMaxLifespanTimestamp(true, true, t, realm) - t);
}
@Test
public void testCalculateUserSessionIdleTimestampOnline() {
long t = Time.currentTimeMillis();
resetRealm();
// non valid, default value
Assert.assertEquals(Constants.DEFAULT_SESSION_IDLE_TIMEOUT * 1000L,
SessionExpirationUtils.calculateUserSessionIdleTimestamp(false, false, t, realm) - t);
// normal value 2000s
realmMap.put("getSsoSessionIdleTimeout", 1000);
Assert.assertEquals(1000 * 1000L, SessionExpirationUtils.calculateUserSessionIdleTimestamp(false, false, t, realm) - t);
// use bigger remember me
realmMap.put("getSsoSessionIdleTimeoutRememberMe", 2000);
Assert.assertEquals(2000 * 1000L, SessionExpirationUtils.calculateUserSessionIdleTimestamp(false, true, t, realm) - t);
}
@Test
public void testCalculateUserSessionIdleTimestampOffline() {
long t = Time.currentTimeMillis();
resetRealm();
// non valid, default value
Assert.assertEquals(Constants.DEFAULT_OFFLINE_SESSION_IDLE_TIMEOUT * 1000L,
SessionExpirationUtils.calculateUserSessionIdleTimestamp(true, false, t, realm) - t);
// normal value 2000s
realmMap.put("getOfflineSessionIdleTimeout", 1000);
Assert.assertEquals(1000 * 1000L, SessionExpirationUtils.calculateUserSessionIdleTimestamp(true, false, t, realm) - t);
// use bigger remember me does not affect
realmMap.put("getSsoSessionIdleTimeoutRememberMe", 2000);
Assert.assertEquals(1000 * 1000L, SessionExpirationUtils.calculateUserSessionIdleTimestamp(true, true, t, realm) - t);
}
@Test
public void testCalculateClientSessionMaxLifespanTimestampOnline() {
long t = Time.currentTimeMillis();
resetRealm();
resetClient();
// default
Assert.assertEquals(Constants.DEFAULT_SESSION_MAX_LIFESPAN * 1000L,
SessionExpirationUtils.calculateClientSessionMaxLifespanTimestamp(false, false, t, t, realm, client) - t);
// normal value in realm
realmMap.put("getSsoSessionMaxLifespan", 5000);
Assert.assertEquals(5000 * 1000L, SessionExpirationUtils.calculateClientSessionMaxLifespanTimestamp(false, false, t, t, realm, client) - t);
// use remember me
realmMap.put("getSsoSessionMaxLifespanRememberMe", 6000);
Assert.assertEquals(6000 * 1000L, SessionExpirationUtils.calculateClientSessionMaxLifespanTimestamp(false, true, t, t, realm, client) - t);
// override client value in realm
realmMap.put("getClientSessionMaxLifespan", 4000);
Assert.assertEquals(4000 * 1000L, SessionExpirationUtils.calculateClientSessionMaxLifespanTimestamp(false, false, t, t, realm, client) - t);
// override value in client
clientMap.put(OIDCConfigAttributes.CLIENT_SESSION_MAX_LIFESPAN, "3000");
Assert.assertEquals(3000 * 1000L, SessionExpirationUtils.calculateClientSessionMaxLifespanTimestamp(false, false, t, t, realm, client) - t);
// client max lifespan cannot be bigger than user lifespan
realmMap.put("getSsoSessionMaxLifespan", 2000);
Assert.assertEquals(2000 * 1000L, SessionExpirationUtils.calculateClientSessionMaxLifespanTimestamp(false, false, t, t, realm, client) - t);
// the same but using remember me
realmMap.put("getSsoSessionMaxLifespan", 1000);
realmMap.put("getSsoSessionMaxLifespanRememberMe", 2000);
Assert.assertEquals(2000 * 1000L, SessionExpirationUtils.calculateClientSessionMaxLifespanTimestamp(false, true, t, t, realm, client) - t);
}
@Test
public void testCalculateClientSessionMaxLifespanTimestampOffline() {
long t = Time.currentTimeMillis();
resetRealm();
resetClient();
// no expiration
Assert.assertEquals(-1, SessionExpirationUtils.calculateClientSessionMaxLifespanTimestamp(true, false, t, t, realm, client));
// default
realmMap.put("isOfflineSessionMaxLifespanEnabled", true);
Assert.assertEquals(Constants.DEFAULT_OFFLINE_SESSION_MAX_LIFESPAN * 1000L,
SessionExpirationUtils.calculateClientSessionMaxLifespanTimestamp(true, false, t, t, realm, client) - t);
// normal value in realm
realmMap.put("getOfflineSessionMaxLifespan", 5000);
Assert.assertEquals(5000 * 1000L, SessionExpirationUtils.calculateClientSessionMaxLifespanTimestamp(true, false, t, t, realm, client) - t);
// override client value in realm
realmMap.put("getClientOfflineSessionMaxLifespan", 4000);
Assert.assertEquals(4000 * 1000L, SessionExpirationUtils.calculateClientSessionMaxLifespanTimestamp(true, false, t, t, realm, client) - t);
// override value in client
clientMap.put(OIDCConfigAttributes.CLIENT_OFFLINE_SESSION_MAX_LIFESPAN, "3000");
Assert.assertEquals(3000 * 1000L, SessionExpirationUtils.calculateClientSessionMaxLifespanTimestamp(true, false, t, t, realm, client) - t);
// client max lifespan cannot be bigger than user lifespan
long t2 = t - 100;
realmMap.put("getOfflineSessionMaxLifespan", 2000);
Assert.assertEquals(2000 * 1000L, SessionExpirationUtils.calculateClientSessionMaxLifespanTimestamp(true, false, t, t2, realm, client) - t2);
}
@Test
public void testCalculateClientSessionIdleTimestampOnline() {
long t = Time.currentTimeMillis();
resetRealm();
resetClient();
// default
Assert.assertEquals(Constants.DEFAULT_SESSION_IDLE_TIMEOUT * 1000L,
SessionExpirationUtils.calculateClientSessionIdleTimestamp(false, false, t, realm, client) - t);
// normal value in realm
realmMap.put("getSsoSessionIdleTimeout", 5000);
Assert.assertEquals(5000 * 1000L, SessionExpirationUtils.calculateClientSessionIdleTimestamp(false, false, t, realm, client) - t);
// use remember me
realmMap.put("getSsoSessionIdleTimeoutRememberMe", 6000);
Assert.assertEquals(6000 * 1000L, SessionExpirationUtils.calculateClientSessionIdleTimestamp(false, true, t, realm, client) - t);
// override client value in realm
realmMap.put("getClientSessionIdleTimeout", 4000);
Assert.assertEquals(4000 * 1000L, SessionExpirationUtils.calculateClientSessionIdleTimestamp(false, false, t, realm, client) - t);
// override value in client
clientMap.put(OIDCConfigAttributes.CLIENT_SESSION_IDLE_TIMEOUT, "3000");
Assert.assertEquals(3000 * 1000L, SessionExpirationUtils.calculateClientSessionIdleTimestamp(false, false, t, realm, client) - t);
}
@Test
public void testCalculateClientSessionIdleTimestampOffline() {
long t = Time.currentTimeMillis();
resetRealm();
resetClient();
// default
Assert.assertEquals(Constants.DEFAULT_OFFLINE_SESSION_IDLE_TIMEOUT * 1000L,
SessionExpirationUtils.calculateClientSessionIdleTimestamp(true, false, t, realm, client) - t);
// normal value in realm
realmMap.put("getOfflineSessionIdleTimeout", 5000);
Assert.assertEquals(5000 * 1000L, SessionExpirationUtils.calculateClientSessionIdleTimestamp(true, false, t, realm, client) - t);
// use remember me does not affect
realmMap.put("getSsoSessionIdleTimeoutRememberMe", 6000);
Assert.assertEquals(5000 * 1000L, SessionExpirationUtils.calculateClientSessionIdleTimestamp(true, true, t, realm, client) - t);
// override client value in realm
realmMap.put("getClientOfflineSessionIdleTimeout", 4000);
Assert.assertEquals(4000 * 1000L, SessionExpirationUtils.calculateClientSessionIdleTimestamp(true, false, t, realm, client) - t);
// override value in client
clientMap.put(OIDCConfigAttributes.CLIENT_OFFLINE_SESSION_IDLE_TIMEOUT, "3000");
Assert.assertEquals(3000 * 1000L, SessionExpirationUtils.calculateClientSessionIdleTimestamp(true, false, t, realm, client) - t);
}
}

View file

@ -27,7 +27,9 @@ import org.keycloak.sessions.CommonClientSessionModel;
*/
public interface AuthenticatedClientSessionModel extends CommonClientSessionModel {
String STARTED_AT_NOTE = "startedAt";
final String STARTED_AT_NOTE = "startedAt";
final String USER_SESSION_STARTED_AT_NOTE = "userSessionStartedAt";
final String USER_SESSION_REMEMBER_ME_NOTE = "userSessionRememberMe";
String getId();
@ -37,6 +39,15 @@ public interface AuthenticatedClientSessionModel extends CommonClientSessionMode
return started == null ? 0 : Integer.parseInt(started);
}
default int getUserSessionStarted() {
String started = getNote(USER_SESSION_STARTED_AT_NOTE);
return started == null ? getUserSession().getStarted() : Integer.parseInt(started);
}
default boolean isUserSessionRememberMe() {
return Boolean.parseBoolean(getNote(USER_SESSION_REMEMBER_ME_NOTE));
}
int getTimestamp();
void setTimestamp(int timestamp);

View file

@ -576,7 +576,7 @@ public class TokenManager {
}
clientSession.setNote(Constants.LEVEL_OF_AUTHENTICATION, String.valueOf(new AcrStore(authSession).getLevelOfAuthenticationFromCurrentAuthentication()));
clientSession.setTimestamp(Time.currentTime());
clientSession.setTimestamp(userSession.getLastSessionRefresh());
// Remove authentication session now
new AuthenticationSessionManager(session).removeAuthenticationSession(userSession.getRealm(), authSession, true);
@ -1129,7 +1129,8 @@ public class TokenManager {
}
if (clientSessionMaxLifespan > 0) {
int clientSessionMaxExpiration = userSession.getStarted() + clientSessionMaxLifespan;
AuthenticatedClientSessionModel clientSession = clientSessionCtx.getClientSession();
int clientSessionMaxExpiration = clientSession.getStarted() + clientSessionMaxLifespan;
sessionExpires = sessionExpires < clientSessionMaxExpiration ? sessionExpires : clientSessionMaxExpiration;
}
@ -1167,7 +1168,8 @@ public class TokenManager {
}
if (clientOfflineSessionMaxLifespan > 0) {
int clientOfflineSessionMaxExpiration = userSession.getStarted() + clientOfflineSessionMaxLifespan;
AuthenticatedClientSessionModel clientSession = clientSessionCtx.getClientSession();
int clientOfflineSessionMaxExpiration = clientSession.getStarted() + clientOfflineSessionMaxLifespan;
sessionExpires = sessionExpires < clientOfflineSessionMaxExpiration ? sessionExpires
: clientOfflineSessionMaxExpiration;
}

View file

@ -312,6 +312,13 @@ public class OAuthClient {
return new AuthorizationEndpointResponse(this);
}
public AuthorizationEndpointResponse doSilentLogin() {
openLoginForm();
WaitUtils.waitForPageToLoad();
return new AuthorizationEndpointResponse(this);
}
public AuthorizationEndpointResponse doLoginSocial(String brokerId, String username, String password) {
openLoginForm();
WaitUtils.waitForPageToLoad();

View file

@ -33,11 +33,16 @@ import org.keycloak.common.constants.ServiceAccountConstants;
import org.keycloak.crypto.Algorithm;
import org.keycloak.events.Details;
import org.keycloak.events.Errors;
import org.keycloak.events.EventType;
import org.keycloak.jose.jws.JWSHeader;
import org.keycloak.jose.jws.JWSInput;
import org.keycloak.models.AdminRoles;
import org.keycloak.models.AuthenticatedClientSessionModel;
import org.keycloak.models.ClientModel;
import org.keycloak.models.Constants;
import org.keycloak.models.KeycloakSession;
import org.keycloak.models.RealmModel;
import org.keycloak.models.UserSessionModel;
import org.keycloak.models.utils.KeycloakModelUtils;
import org.keycloak.models.utils.SessionTimeoutHelper;
import org.keycloak.protocol.oidc.OIDCConfigAttributes;
@ -78,10 +83,10 @@ import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.keycloak.testsuite.Assert.assertExpiration;
import static org.keycloak.testsuite.admin.AbstractAdminTest.loadJson;
import static org.keycloak.testsuite.admin.ApiUtil.findRealmRoleByName;
@ -710,13 +715,16 @@ public class OfflineTokenTest extends AbstractKeycloakTest {
}
// KEYCLOAK-7688 Offline Session Max for Offline Token
private int[] changeOfflineSessionSettings(boolean isEnabled, int sessionMax, int sessionIdle) {
int prev[] = new int[2];
private int[] changeOfflineSessionSettings(boolean isEnabled, int sessionMax, int sessionIdle, int clientSessionMax, int clientSessionIdle) {
int prev[] = new int[5];
RealmRepresentation rep = adminClient.realm("test").toRepresentation();
prev[0] = rep.getOfflineSessionMaxLifespan().intValue();
prev[1] = rep.getOfflineSessionIdleTimeout().intValue();
prev[0] = rep.getOfflineSessionMaxLifespan();
prev[1] = rep.getOfflineSessionIdleTimeout();
prev[2] = rep.getClientOfflineSessionMaxLifespan();
prev[3] = rep.getClientOfflineSessionIdleTimeout();
RealmBuilder realmBuilder = RealmBuilder.create();
realmBuilder.offlineSessionMaxLifespanEnabled(isEnabled).offlineSessionMaxLifespan(sessionMax).offlineSessionIdleTimeout(sessionIdle);
realmBuilder.offlineSessionMaxLifespanEnabled(isEnabled).offlineSessionMaxLifespan(sessionMax).offlineSessionIdleTimeout(sessionIdle)
.clientOfflineSessionMaxLifespan(clientSessionMax).clientOfflineSessionIdleTimeout(clientSessionIdle);
adminClient.realm("test").update(realmBuilder.build());
return prev;
}
@ -724,8 +732,8 @@ public class OfflineTokenTest extends AbstractKeycloakTest {
private int[] changeSessionSettings(int ssoSessionIdle, int accessTokenLifespan) {
int prev[] = new int[2];
RealmRepresentation rep = adminClient.realm("test").toRepresentation();
prev[0] = rep.getOfflineSessionMaxLifespan().intValue();
prev[1] = rep.getOfflineSessionIdleTimeout().intValue();
prev[0] = rep.getOfflineSessionMaxLifespan();
prev[1] = rep.getOfflineSessionIdleTimeout();
RealmBuilder realmBuilder = RealmBuilder.create();
realmBuilder.ssoSessionIdleTimeout(ssoSessionIdle).accessTokenLifespan(accessTokenLifespan);
adminClient.realm("test").update(realmBuilder.build());
@ -737,7 +745,7 @@ public class OfflineTokenTest extends AbstractKeycloakTest {
// expect that offline session expired by max lifespan
final int MAX_LIFESPAN = 3600;
final int IDLE_LIFESPAN = 6000;
testOfflineSessionExpiration(IDLE_LIFESPAN, MAX_LIFESPAN, MAX_LIFESPAN + 60);
testOfflineSessionExpiration(IDLE_LIFESPAN, MAX_LIFESPAN, MAX_LIFESPAN / 2, MAX_LIFESPAN + 60);
}
@Test
@ -746,7 +754,7 @@ public class OfflineTokenTest extends AbstractKeycloakTest {
final int MAX_LIFESPAN = 3000;
final int IDLE_LIFESPAN = 600;
// Additional time window is added for the case when session was updated in different DC and the update to current DC was postponed
testOfflineSessionExpiration(IDLE_LIFESPAN, MAX_LIFESPAN, IDLE_LIFESPAN + SessionTimeoutHelper.IDLE_TIMEOUT_WINDOW_SECONDS + 60);
testOfflineSessionExpiration(IDLE_LIFESPAN, MAX_LIFESPAN, 0, IDLE_LIFESPAN + SessionTimeoutHelper.IDLE_TIMEOUT_WINDOW_SECONDS + 60);
}
// Issue 13706
@ -761,7 +769,7 @@ public class OfflineTokenTest extends AbstractKeycloakTest {
int prev[] = null;
try (RealmAttributeUpdater rau = new RealmAttributeUpdater(adminClient.realm("test")).setSsoSessionIdleTimeout(900).update()) {
// Step 1 - offline login with "offline-client"
prev = changeOfflineSessionSettings(true, MAX_LIFESPAN, IDLE_LIFESPAN);
prev = changeOfflineSessionSettings(true, MAX_LIFESPAN, IDLE_LIFESPAN, 0, 0);
oauth.scope(OAuth2Constants.OFFLINE_ACCESS);
oauth.clientId("offline-client");
@ -794,7 +802,7 @@ public class OfflineTokenTest extends AbstractKeycloakTest {
} finally {
getTestingClient().testing().revertTestingInfinispanTimeService();
changeOfflineSessionSettings(false, prev[0], prev[1]);
changeOfflineSessionSettings(false, prev[0], prev[1], 0, 0);
}
}
@ -829,10 +837,35 @@ public class OfflineTokenTest extends AbstractKeycloakTest {
}
}
private void testOfflineSessionExpiration(int idleTime, int maxLifespan, int offset) {
private String getOfflineClientSessionUuid(final String userSessionId, final String clientId) {
return testingClient.server().fetch(session -> {
RealmModel realmModel = session.realms().getRealmByName("test");
ClientModel clientModel = realmModel.getClientByClientId(clientId);
UserSessionModel userSession = session.sessions().getOfflineUserSession(realmModel, userSessionId);
AuthenticatedClientSessionModel clientSession = userSession.getAuthenticatedClientSessionByClient(clientModel.getId());
return clientSession.getId();
}, String.class);
}
private int checkIfUserAndClientSessionExist(final String userSessionId, final String clientId, final String clientSessionId) {
return testingClient.server().fetch(session -> {
RealmModel realmModel = session.realms().getRealmByName("test");
session.getContext().setRealm(realmModel);
ClientModel clientModel = realmModel.getClientByClientId(clientId);
UserSessionModel userSession = session.sessions().getOfflineUserSession(realmModel, userSessionId);
if (userSession != null) {
AuthenticatedClientSessionModel clientSession = userSession.getAuthenticatedClientSessionByClient(clientModel.getId());
return clientSession != null && clientSessionId.equals(clientSession.getId())? 2 : 1;
}
return 0;
}, Integer.class);
}
private void testOfflineSessionExpiration(int idleTime, int maxLifespan, int offsetHalf, int offset) {
int prev[] = null;
getTestingClient().testing().setTestingInfinispanTimeService();
try {
prev = changeOfflineSessionSettings(true, maxLifespan, idleTime);
prev = changeOfflineSessionSettings(true, maxLifespan, idleTime, 0, 0);
oauth.scope(OAuth2Constants.OFFLINE_ACCESS);
oauth.clientId("offline-client");
@ -854,12 +887,20 @@ public class OfflineTokenTest extends AbstractKeycloakTest {
assertEquals(TokenUtil.TOKEN_TYPE_OFFLINE, offlineToken.getType());
// obtain the client session ID
final String clientSessionId = getOfflineClientSessionUuid(sessionId, loginEvent.getClientId());
assertEquals(2, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
// perform a refresh in the half-time
setTimeOffset(offsetHalf);
tokenResponse = oauth.doRefreshTokenRequest(offlineTokenString, "secret1");
AccessToken refreshedToken = oauth.verifyToken(tokenResponse.getAccessToken());
offlineTokenString = tokenResponse.getRefreshToken();
offlineToken = oauth.parseRefreshToken(offlineTokenString);
Assert.assertEquals(200, tokenResponse.getStatusCode());
assertEquals(2, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
// wait to expire
setTimeOffset(offset);
@ -870,6 +911,7 @@ public class OfflineTokenTest extends AbstractKeycloakTest {
assertEquals("invalid_grant", tokenResponse.getError());
// Assert userSession expired
assertEquals(0, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
testingClient.testing().removeExpired("test");
try {
testingClient.testing().removeUserSession("test", sessionId);
@ -880,7 +922,8 @@ public class OfflineTokenTest extends AbstractKeycloakTest {
setTimeOffset(0);
} finally {
changeOfflineSessionSettings(false, prev[0], prev[1]);
getTestingClient().testing().revertTestingInfinispanTimeService();
changeOfflineSessionSettings(false, prev[0], prev[1], prev[2], prev[3]);
}
}
@ -954,12 +997,106 @@ public class OfflineTokenTest extends AbstractKeycloakTest {
}
@Test
public void refreshTokenUserClientMaxLifespanSmallerThanSession() throws Exception {
oauth.scope(OAuth2Constants.OFFLINE_ACCESS);
oauth.clientId("offline-client");
oauth.redirectUri(offlineClientAppUri);
int[] prev = changeOfflineSessionSettings(true, 3600, 7200, 1000, 7200);
getTestingClient().testing().setTestingInfinispanTimeService();
try {
oauth.doLogin("test-user@localhost", "password");
EventRepresentation loginEvent = events.expectLogin().client("offline-client")
.detail(Details.REDIRECT_URI, offlineClientAppUri).assertEvent();
String sessionId = loginEvent.getSessionId();
String code = oauth.getCurrentQuery().get(OAuth2Constants.CODE);
OAuthClient.AccessTokenResponse tokenResponse = oauth.doAccessTokenRequest(code, "secret1");
assertEquals(TokenUtil.TOKEN_TYPE_OFFLINE, oauth.parseRefreshToken(tokenResponse.getRefreshToken()).getType());
assertTrue("Invalid ExpiresIn", 0 < tokenResponse.getRefreshExpiresIn() && tokenResponse.getRefreshExpiresIn() <= 1000);
String clientSessionId = getOfflineClientSessionUuid(sessionId, loginEvent.getClientId());
assertEquals(2, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
events.poll();
setTimeOffset(600);
String refreshId = oauth.parseRefreshToken(tokenResponse.getRefreshToken()).getId();
tokenResponse = oauth.doRefreshTokenRequest(tokenResponse.getRefreshToken(), "secret1");
assertEquals(TokenUtil.TOKEN_TYPE_OFFLINE, oauth.parseRefreshToken(tokenResponse.getRefreshToken()).getType());
assertTrue("Invalid ExpiresIn", 0 < tokenResponse.getRefreshExpiresIn() && tokenResponse.getRefreshExpiresIn() <= 400);
assertEquals(2, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
events.expectRefresh(refreshId, sessionId).client("offline-client").detail(Details.REFRESH_TOKEN_TYPE, TokenUtil.TOKEN_TYPE_OFFLINE).assertEvent();
setTimeOffset(1100);
tokenResponse = oauth.doRefreshTokenRequest(tokenResponse.getRefreshToken(), "secret1");
assertEquals(400, tokenResponse.getStatusCode());
assertNull(tokenResponse.getAccessToken());
assertNull(tokenResponse.getRefreshToken());
events.expect(EventType.REFRESH_TOKEN).client("offline-client").error(Errors.INVALID_TOKEN).user((String) null).assertEvent();
assertEquals(1, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
} finally {
changeOfflineSessionSettings(false, prev[0], prev[1], prev[2], prev[3]);
getTestingClient().testing().revertTestingInfinispanTimeService();
events.clear();
resetTimeOffset();
}
}
@Test
public void refreshTokenUserClientMaxLifespanGreaterThanSession() throws Exception {
oauth.scope(OAuth2Constants.OFFLINE_ACCESS);
oauth.clientId("offline-client");
oauth.redirectUri(offlineClientAppUri);
int[] prev = changeOfflineSessionSettings(true, 3600, 7200, 5000, 7200);
getTestingClient().testing().setTestingInfinispanTimeService();
try {
oauth.doLogin("test-user@localhost", "password");
EventRepresentation loginEvent = events.expectLogin().client("offline-client")
.detail(Details.REDIRECT_URI, offlineClientAppUri).assertEvent();
String sessionId = loginEvent.getSessionId();
String code = oauth.getCurrentQuery().get(OAuth2Constants.CODE);
OAuthClient.AccessTokenResponse tokenResponse = oauth.doAccessTokenRequest(code, "secret1");
assertEquals(TokenUtil.TOKEN_TYPE_OFFLINE, oauth.parseRefreshToken(tokenResponse.getRefreshToken()).getType());
assertTrue("Invalid ExpiresIn", 0 < tokenResponse.getRefreshExpiresIn() && tokenResponse.getRefreshExpiresIn() <= 3600);
String clientSessionId = getOfflineClientSessionUuid(sessionId, loginEvent.getClientId());
assertEquals(2, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
events.poll();
setTimeOffset(1800);
String refreshId = oauth.parseRefreshToken(tokenResponse.getRefreshToken()).getId();
tokenResponse = oauth.doRefreshTokenRequest(tokenResponse.getRefreshToken(), "secret1");
assertEquals(TokenUtil.TOKEN_TYPE_OFFLINE, oauth.parseRefreshToken(tokenResponse.getRefreshToken()).getType());
assertTrue("Invalid ExpiresIn", 0 < tokenResponse.getRefreshExpiresIn() && tokenResponse.getRefreshExpiresIn() <= 1800);
assertEquals(2, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
events.expectRefresh(refreshId, sessionId).client("offline-client").detail(Details.REFRESH_TOKEN_TYPE, TokenUtil.TOKEN_TYPE_OFFLINE).assertEvent();
setTimeOffset(3700);
tokenResponse = oauth.doRefreshTokenRequest(tokenResponse.getRefreshToken(), "secret1");
assertEquals(400, tokenResponse.getStatusCode());
assertNull(tokenResponse.getAccessToken());
assertNull(tokenResponse.getRefreshToken());
events.expect(EventType.REFRESH_TOKEN).client("offline-client").error(Errors.INVALID_TOKEN).user((String) null).assertEvent();
assertEquals(0, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
} finally {
changeOfflineSessionSettings(false, prev[0], prev[1], prev[2], prev[3]);
getTestingClient().testing().revertTestingInfinispanTimeService();
events.clear();
resetTimeOffset();
}
}
@Test
public void testShortOfflineSessionMax() throws Exception {
int prevOfflineSession[] = null;
int prevSession[] = null;
try {
prevOfflineSession = changeOfflineSessionSettings(true, 60, 30);
prevOfflineSession = changeOfflineSessionSettings(true, 60, 30, 0, 0);
prevSession = changeSessionSettings(1800, 300);
oauth.scope(OAuth2Constants.OFFLINE_ACCESS);
@ -989,7 +1126,7 @@ public class OfflineTokenTest extends AbstractKeycloakTest {
allOf(greaterThanOrEqualTo(59), lessThanOrEqualTo(60)));
} finally {
changeOfflineSessionSettings(false, prevOfflineSession[0], prevOfflineSession[1]);
changeOfflineSessionSettings(false, prevOfflineSession[0], prevOfflineSession[1], prevOfflineSession[2], prevOfflineSession[3]);
changeSessionSettings(prevSession[0], prevSession[1]);
}
}

View file

@ -18,6 +18,7 @@ package org.keycloak.testsuite.oauth;
import com.fasterxml.jackson.databind.JsonNode;
import com.gargoylesoftware.htmlunit.WebClient;
import java.io.Closeable;
import org.hamcrest.CoreMatchers;
import org.jboss.arquillian.drone.webdriver.htmlunit.DroneHtmlUnitDriver;
import org.jboss.arquillian.graphene.page.Page;
@ -33,12 +34,16 @@ import org.keycloak.admin.client.resource.RealmsResource;
import org.keycloak.admin.client.resource.UserResource;
import org.keycloak.common.enums.SslRequired;
import org.keycloak.crypto.Algorithm;
import org.keycloak.events.EventType;
import org.keycloak.events.Details;
import org.keycloak.events.Errors;
import org.keycloak.jose.jws.JWSHeader;
import org.keycloak.jose.jws.JWSInput;
import org.keycloak.models.AuthenticatedClientSessionModel;
import org.keycloak.models.ClientModel;
import org.keycloak.models.RealmModel;
import org.keycloak.models.UserModel;
import org.keycloak.models.UserSessionModel;
import org.keycloak.models.utils.KeycloakModelUtils;
import org.keycloak.models.utils.SessionTimeoutHelper;
import org.keycloak.protocol.oidc.OIDCConfigAttributes;
@ -1045,37 +1050,182 @@ public class RefreshTokenTest extends AbstractKeycloakTest {
}
}
private String getClientSessionUuid(final String userSessionId, String clientId) {
return testingClient.server().fetch(session -> {
RealmModel realmModel = session.realms().getRealmByName("test");
ClientModel clientModel = realmModel.getClientByClientId(clientId);
UserSessionModel userSession = session.sessions().getUserSession(realmModel, userSessionId);
AuthenticatedClientSessionModel clientSession = userSession.getAuthenticatedClientSessionByClient(clientModel.getId());
return clientSession.getId();
}, String.class);
}
private int checkIfUserAndClientSessionExist(final String userSessionId, final String clientId, final String clientSessionId) {
return testingClient.server().fetch(session -> {
RealmModel realmModel = session.realms().getRealmByName("test");
ClientModel clientModel = realmModel.getClientByClientId(clientId);
UserSessionModel userSession = session.sessions().getUserSession(realmModel, userSessionId);
if (userSession != null) {
AuthenticatedClientSessionModel clientSession = userSession.getAuthenticatedClientSessionByClient(clientModel.getId());
return clientSession != null && clientSessionId.equals(clientSession.getId())? 2 : 1;
}
return 0;
}, Integer.class);
}
@Test
public void refreshTokenUserSessionMaxLifespan() throws Exception {
RealmResource realmResource = adminClient.realm("test");
getTestingClient().testing().setTestingInfinispanTimeService();
try (Closeable realmUpdater = new RealmAttributeUpdater(realmResource)
.updateWith(r -> {
r.setSsoSessionMaxLifespan(3600);
r.setSsoSessionIdleTimeout(7200);
}).update()) {
oauth.doLogin("test-user@localhost", "password");
EventRepresentation loginEvent = events.expectLogin().assertEvent();
String sessionId = loginEvent.getSessionId();
String code = oauth.getCurrentQuery().get(OAuth2Constants.CODE);
OAuthClient.AccessTokenResponse tokenResponse = oauth.doAccessTokenRequest(code, "password");
assertTrue("Invalid ExpiresIn", 0 < tokenResponse.getRefreshExpiresIn() && tokenResponse.getRefreshExpiresIn() <= 3600);
final String clientSessionId = getClientSessionUuid(sessionId, loginEvent.getClientId());
assertEquals(2, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
events.poll();
setTimeOffset(1800);
String refreshId = oauth.parseRefreshToken(tokenResponse.getRefreshToken()).getId();
tokenResponse = oauth.doRefreshTokenRequest(tokenResponse.getRefreshToken(), "password");
assertTrue("Invalid ExpiresIn", 0 < tokenResponse.getRefreshExpiresIn() && tokenResponse.getRefreshExpiresIn() <= 1800);
assertEquals(2, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
events.expectRefresh(refreshId, sessionId).assertEvent();
RealmResource realmResource = adminClient.realm("test");
Integer maxLifespan = realmResource.toRepresentation().getSsoSessionMaxLifespan();
try {
RealmManager.realm(realmResource).ssoSessionMaxLifespan(1);
setTimeOffset(2);
setTimeOffset(3700);
oauth.parseRefreshToken(tokenResponse.getRefreshToken()).getId();
tokenResponse = oauth.doRefreshTokenRequest(tokenResponse.getRefreshToken(), "password");
assertEquals(400, tokenResponse.getStatusCode());
assertNull(tokenResponse.getAccessToken());
assertNull(tokenResponse.getRefreshToken());
events.expectRefresh(refreshId, sessionId).error(Errors.INVALID_TOKEN);
events.expect(EventType.REFRESH_TOKEN).error(Errors.INVALID_TOKEN).user((String) null).assertEvent();
assertEquals(0, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
} finally {
RealmManager.realm(realmResource).ssoSessionMaxLifespan(maxLifespan);
getTestingClient().testing().revertTestingInfinispanTimeService();
events.clear();
resetTimeOffset();
}
}
@Test
public void refreshTokenUserClientMaxLifespanSmallerThanSession() throws Exception {
RealmResource realmResource = adminClient.realm("test");
getTestingClient().testing().setTestingInfinispanTimeService();
try (Closeable realmUpdater = new RealmAttributeUpdater(realmResource)
.updateWith(r -> {
r.setSsoSessionMaxLifespan(3600);
r.setSsoSessionIdleTimeout(7200);
r.setClientSessionMaxLifespan(1000);
r.setClientSessionIdleTimeout(7200);
}).update()) {
oauth.doLogin("test-user@localhost", "password");
EventRepresentation loginEvent = events.expectLogin().assertEvent();
String sessionId = loginEvent.getSessionId();
String code = oauth.getCurrentQuery().get(OAuth2Constants.CODE);
OAuthClient.AccessTokenResponse tokenResponse = oauth.doAccessTokenRequest(code, "password");
assertTrue("Invalid ExpiresIn", 0 < tokenResponse.getRefreshExpiresIn() && tokenResponse.getRefreshExpiresIn() <= 1000);
String clientSessionId = getClientSessionUuid(sessionId, loginEvent.getClientId());
assertEquals(2, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
events.poll();
setTimeOffset(600);
String refreshId = oauth.parseRefreshToken(tokenResponse.getRefreshToken()).getId();
tokenResponse = oauth.doRefreshTokenRequest(tokenResponse.getRefreshToken(), "password");
assertTrue("Invalid ExpiresIn", 0 < tokenResponse.getRefreshExpiresIn() && tokenResponse.getRefreshExpiresIn() <= 400);
assertEquals(2, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
events.expectRefresh(refreshId, sessionId).assertEvent();
setTimeOffset(1100);
tokenResponse = oauth.doRefreshTokenRequest(tokenResponse.getRefreshToken(), "password");
assertEquals(400, tokenResponse.getStatusCode());
assertNull(tokenResponse.getAccessToken());
assertNull(tokenResponse.getRefreshToken());
events.expect(EventType.REFRESH_TOKEN).error(Errors.INVALID_TOKEN).user((String) null).assertEvent();
assertEquals(1, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
setTimeOffset(1600);
oauth.doSilentLogin();
loginEvent = events.expectLogin().assertEvent();
sessionId = loginEvent.getSessionId();
code = oauth.getCurrentQuery().get(OAuth2Constants.CODE);
tokenResponse = oauth.doAccessTokenRequest(code, "password");
assertTrue("Invalid ExpiresIn", 0 < tokenResponse.getRefreshExpiresIn() && tokenResponse.getRefreshExpiresIn() <= 1000);
events.expectCodeToToken(loginEvent.getDetails().get(Details.CODE_ID), sessionId).assertEvent();
clientSessionId = getClientSessionUuid(sessionId, loginEvent.getClientId());
assertEquals(2, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
setTimeOffset(3700);
tokenResponse = oauth.doRefreshTokenRequest(tokenResponse.getRefreshToken(), "password");
assertEquals(400, tokenResponse.getStatusCode());
assertNull(tokenResponse.getAccessToken());
assertNull(tokenResponse.getRefreshToken());
events.expect(EventType.REFRESH_TOKEN).error(Errors.INVALID_TOKEN).user((String) null).assertEvent();
assertEquals(0, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
} finally {
getTestingClient().testing().revertTestingInfinispanTimeService();
events.clear();
resetTimeOffset();
}
}
@Test
public void refreshTokenUserClientMaxLifespanGreaterThanSession() throws Exception {
RealmResource realmResource = adminClient.realm("test");
getTestingClient().testing().setTestingInfinispanTimeService();
try (Closeable realmUpdater = new RealmAttributeUpdater(realmResource)
.updateWith(r -> {
r.setSsoSessionMaxLifespan(3600);
r.setSsoSessionIdleTimeout(7200);
r.setClientSessionMaxLifespan(5000);
r.setClientSessionIdleTimeout(7200);
}).update()) {
oauth.doLogin("test-user@localhost", "password");
EventRepresentation loginEvent = events.expectLogin().assertEvent();
String sessionId = loginEvent.getSessionId();
String code = oauth.getCurrentQuery().get(OAuth2Constants.CODE);
OAuthClient.AccessTokenResponse tokenResponse = oauth.doAccessTokenRequest(code, "password");
assertTrue("Invalid ExpiresIn", 0 < tokenResponse.getRefreshExpiresIn() && tokenResponse.getRefreshExpiresIn() <= 3600);
String clientSessionId = getClientSessionUuid(sessionId, loginEvent.getClientId());
assertEquals(2, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
events.poll();
setTimeOffset(1800);
String refreshId = oauth.parseRefreshToken(tokenResponse.getRefreshToken()).getId();
tokenResponse = oauth.doRefreshTokenRequest(tokenResponse.getRefreshToken(), "password");
assertTrue("Invalid ExpiresIn", 0 < tokenResponse.getRefreshExpiresIn() && tokenResponse.getRefreshExpiresIn() <= 1800);
assertEquals(2, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
events.expectRefresh(refreshId, sessionId).assertEvent();
setTimeOffset(3700);
tokenResponse = oauth.doRefreshTokenRequest(tokenResponse.getRefreshToken(), "password");
assertEquals(400, tokenResponse.getStatusCode());
assertNull(tokenResponse.getAccessToken());
assertNull(tokenResponse.getRefreshToken());
events.expect(EventType.REFRESH_TOKEN).error(Errors.INVALID_TOKEN).user((String) null).assertEvent();
assertEquals(0, checkIfUserAndClientSessionExist(sessionId, loginEvent.getClientId(), clientSessionId));
} finally {
getTestingClient().testing().revertTestingInfinispanTimeService();
events.clear();
resetTimeOffset();
}
@ -1132,6 +1282,79 @@ public class RefreshTokenTest extends AbstractKeycloakTest {
}
}
@Test
public void refreshTokenClientSessionMaxLifespan() throws Exception {
RealmResource realm = adminClient.realm("test");
RealmRepresentation rep = realm.toRepresentation();
Integer originalSsoSessionMaxLifespan = rep.getSsoSessionMaxLifespan();
ClientResource client = ApiUtil.findClientByClientId(adminClient.realm("test"), "test-app");
ClientRepresentation clientRepresentation = client.toRepresentation();
getTestingClient().testing().setTestingInfinispanTimeService();
try {
rep.setSsoSessionMaxLifespan(1000);
realm.update(rep);
clientRepresentation.getAttributes().put(OIDCConfigAttributes.CLIENT_SESSION_MAX_LIFESPAN, "500");
client.update(clientRepresentation);
oauth.doLogin("test-user@localhost", "password");
EventRepresentation loginEvent = events.expectLogin().assertEvent();
String sessionId = loginEvent.getSessionId();
String code = oauth.getCurrentQuery().get(OAuth2Constants.CODE);
OAuthClient.AccessTokenResponse tokenResponse = oauth.doAccessTokenRequest(code, "password");
events.poll();
String refreshId = oauth.parseRefreshToken(tokenResponse.getRefreshToken()).getId();
tokenResponse = oauth.doRefreshTokenRequest(tokenResponse.getRefreshToken(), "password");
assertTrue("Invalid RefreshExpiresIn", 0 < tokenResponse.getRefreshExpiresIn() && tokenResponse.getRefreshExpiresIn() <= 500);
setTimeOffset(100);
tokenResponse = oauth.doRefreshTokenRequest(tokenResponse.getRefreshToken(), "password");
assertTrue("Invalid RefreshExpiresIn", 0 < tokenResponse.getRefreshExpiresIn() && tokenResponse.getRefreshExpiresIn() <= 400);
setTimeOffset(600);
oauth.doSilentLogin();
code = oauth.getCurrentQuery().get(OAuth2Constants.CODE);
tokenResponse = oauth.doAccessTokenRequest(code, "password");
assertEquals(200, tokenResponse.getStatusCode());
assertTrue("Invalid RefreshExpiresIn", 0 < tokenResponse.getRefreshExpiresIn() && tokenResponse.getRefreshExpiresIn() <= 400);
setTimeOffset(700);
tokenResponse = oauth.doRefreshTokenRequest(tokenResponse.getRefreshToken(), "password");
assertEquals(200, tokenResponse.getStatusCode());
assertTrue("Invalid RefreshExpiresIn", 0 < tokenResponse.getRefreshExpiresIn() && tokenResponse.getRefreshExpiresIn() <= 300);
setTimeOffset(1100);
tokenResponse = oauth.doRefreshTokenRequest(tokenResponse.getRefreshToken(), "password");
assertEquals(400, tokenResponse.getStatusCode());
assertNull(tokenResponse.getAccessToken());
assertNull(tokenResponse.getRefreshToken());
events.expectRefresh(refreshId, sessionId).error(Errors.INVALID_TOKEN);
} finally {
rep.setSsoSessionMaxLifespan(originalSsoSessionMaxLifespan);
realm.update(rep);
clientRepresentation.getAttributes().put(OIDCConfigAttributes.CLIENT_SESSION_MAX_LIFESPAN, null);
client.update(clientRepresentation);
events.clear();
resetTimeOffset();
getTestingClient().testing().revertTestingInfinispanTimeService();
}
}
@Test
public void testCheckSsl() throws Exception {
Client client = AdminClientUtil.createResteasyClient();

View file

@ -172,4 +172,18 @@ public class RealmManager {
realm.update(rep);
return this;
}
public RealmManager clientSessionMaxLifespan(int clientSessionLaxLifespan) {
RealmRepresentation rep = realm.toRepresentation();
rep.setClientSessionMaxLifespan(clientSessionLaxLifespan);
realm.update(rep);
return this;
}
public RealmManager clientSessionIdleTimeout(int clientSessionIdleTimeout) {
RealmRepresentation rep = realm.toRepresentation();
rep.setClientSessionIdleTimeout(clientSessionIdleTimeout);
realm.update(rep);
return this;
}
}

View file

@ -19,9 +19,7 @@ package org.keycloak.testsuite.ui.account2;
import org.jboss.arquillian.graphene.page.Page;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
import org.keycloak.models.utils.SessionTimeoutHelper;
import org.keycloak.representations.idm.RealmRepresentation;
import org.keycloak.testsuite.ui.account2.page.DeviceActivityPage;
import org.keycloak.testsuite.ui.account2.page.PersonalInfoPage;
@ -36,7 +34,7 @@ import static org.keycloak.testsuite.util.WaitUtils.pause;
* @author Vaclav Muzikar <vmuzikar@redhat.com>
*/
public class SessionTest extends AbstractAccountTest {
public static final int SSO_SESSION_IDLE_TIMEOUT = 1;
public static final int SSO_SESSION_IDLE_TIMEOUT = 10;
public static final int ACCESS_TOKEN_LIFESPAN = 10;
@Page
@ -51,8 +49,8 @@ public class SessionTest extends AbstractAccountTest {
RealmRepresentation realm = testRealms.get(0);
// in seconds
realm.setSsoSessionIdleTimeout(1);
realm.setAccessTokenLifespan(10);
realm.setSsoSessionIdleTimeout(SSO_SESSION_IDLE_TIMEOUT);
realm.setAccessTokenLifespan(ACCESS_TOKEN_LIFESPAN);
}
@Before
@ -107,6 +105,6 @@ public class SessionTest extends AbstractAccountTest {
private void waitForSessionToExpire() {
// +3 to add some toleration
log.info("Waiting for SSO session to expire");
pause((SSO_SESSION_IDLE_TIMEOUT + SessionTimeoutHelper.IDLE_TIMEOUT_WINDOW_SECONDS + 3) * 1000);
pause((SSO_SESSION_IDLE_TIMEOUT + 3) * 1000);
}
}

View file

@ -0,0 +1,360 @@
/*
* Copyright 2023 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.session;
import java.util.UUID;
import org.junit.Assert;
import org.junit.FixMethodOrder;
import org.junit.Test;
import org.junit.runners.MethodSorters;
import org.keycloak.common.util.Time;
import org.keycloak.models.AuthenticatedClientSessionModel;
import org.keycloak.models.ClientModel;
import org.keycloak.models.Constants;
import org.keycloak.models.KeycloakSession;
import org.keycloak.models.RealmModel;
import org.keycloak.models.RealmProvider;
import org.keycloak.models.UserModel;
import org.keycloak.models.UserSessionModel;
import org.keycloak.models.UserProvider;
import org.keycloak.models.UserSessionProvider;
import org.keycloak.protocol.oidc.OIDCConfigAttributes;
import org.keycloak.protocol.oidc.OIDCLoginProtocol;
import org.keycloak.testsuite.model.KeycloakModelTest;
import org.keycloak.testsuite.model.RequireProvider;
import org.keycloak.testsuite.model.infinispan.InfinispanTestUtil;
/**
* <p>
* Test that checks the Infinispan user session provider expires the sessions
* correctly and does not remain client sessions in memory after user session
* expiration.</p>
*
* @author rmartinc
*/
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
@RequireProvider(UserSessionProvider.class)
@RequireProvider(UserProvider.class)
@RequireProvider(RealmProvider.class)
public class SessionTimeoutsTest extends KeycloakModelTest {
private String realmId;
@Override
public void createEnvironment(KeycloakSession s) {
RealmModel realm = createRealm(s, "test");
realm.setDefaultRole(s.roles().addRealmRole(realm, Constants.DEFAULT_ROLES_ROLE_PREFIX + "-" + realm.getName()));
this.realmId = realm.getId();
s.users().addUser(realm, "user1").setEmail("user1@localhost");
s.clients().addClient(realm, "test-app");
InfinispanTestUtil.setTestingTimeService(s);
}
@Override
public void cleanEnvironment(KeycloakSession s) {
InfinispanTestUtil.revertTimeService(s);
RealmModel realm = s.realms().getRealm(realmId);
UserModel user1 = s.users().getUserByUsername(realm, "user1");
s.sessions().removeUserSessions(realm);
s.sessions().getOfflineUserSessionsStream(realm, user1).forEach(us -> s.sessions().removeOfflineUserSession(realm, us));
s.realms().removeRealm(realmId);
}
protected static UserSessionModel createUserSession(KeycloakSession session, RealmModel realm, UserModel user, boolean offline) {
UserSessionModel userSession = session.sessions().createUserSession(UUID.randomUUID().toString(), realm, user, "user1", "127.0.0.1",
"form", true, null, null, UserSessionModel.SessionPersistenceState.PERSISTENT);
if (offline) {
userSession = session.sessions().createOfflineUserSession(userSession);
}
return userSession;
}
protected static AuthenticatedClientSessionModel createClientSession(KeycloakSession session, String realmId, ClientModel client,
UserSessionModel userSession, String redirect, String state) {
RealmModel realm = session.realms().getRealm(realmId);
AuthenticatedClientSessionModel clientSession = session.sessions().createClientSession(realm, client, userSession);
if (userSession.isOffline()) {
clientSession = session.sessions().createOfflineClientSession(clientSession, userSession);
}
clientSession.setRedirectUri(redirect);
if (state != null) {
clientSession.setNote(OIDCLoginProtocol.STATE_PARAM, state);
}
return clientSession;
}
protected static UserSessionModel getUserSession(KeycloakSession session, RealmModel realm, String id, boolean offline) {
return offline
? session.sessions().getOfflineUserSession(realm, id)
: session.sessions().getUserSession(realm, id);
}
protected void configureTimeouts(int realmMaxLifespan, int realmIdleTimeout, boolean overrideInClient, boolean lifespan, int clientValue) {
withRealm(realmId, (session, realm) -> {
realm.setOfflineSessionMaxLifespanEnabled(true);
realm.setOfflineSessionMaxLifespan(realmMaxLifespan);
realm.setOfflineSessionIdleTimeout(realmIdleTimeout);
realm.setClientOfflineSessionMaxLifespan(realmMaxLifespan);
realm.setClientOfflineSessionIdleTimeout(realmIdleTimeout);
realm.setSsoSessionMaxLifespan(realmMaxLifespan);
realm.setSsoSessionIdleTimeout(realmIdleTimeout);
realm.setClientSessionMaxLifespan(realmMaxLifespan);
realm.setClientSessionIdleTimeout(realmIdleTimeout);
String clientValueString = Integer.toString(clientValue);
ClientModel client = realm.getClientByClientId("test-app");
client.removeAttribute(OIDCConfigAttributes.CLIENT_OFFLINE_SESSION_MAX_LIFESPAN);
client.removeAttribute(OIDCConfigAttributes.CLIENT_OFFLINE_SESSION_IDLE_TIMEOUT);
client.removeAttribute(OIDCConfigAttributes.CLIENT_SESSION_MAX_LIFESPAN);
client.removeAttribute(OIDCConfigAttributes.CLIENT_SESSION_IDLE_TIMEOUT);
if (overrideInClient) {
if (lifespan) {
client.setAttribute(OIDCConfigAttributes.CLIENT_OFFLINE_SESSION_MAX_LIFESPAN, clientValueString);
client.setAttribute(OIDCConfigAttributes.CLIENT_SESSION_MAX_LIFESPAN, clientValueString);
} else {
client.setAttribute(OIDCConfigAttributes.CLIENT_OFFLINE_SESSION_IDLE_TIMEOUT, clientValueString);
client.setAttribute(OIDCConfigAttributes.CLIENT_SESSION_IDLE_TIMEOUT, clientValueString);
}
} else {
if (lifespan) {
realm.setClientOfflineSessionMaxLifespan(clientValue);
realm.setClientSessionMaxLifespan(clientValue);
} else {
realm.setClientOfflineSessionIdleTimeout(clientValue);
realm.setClientSessionIdleTimeout(clientValue);
}
}
return null;
});
}
protected void testUserClientMaxLifespanSmallerThanSession(boolean offline, boolean overrideInClient) {
configureTimeouts(3000, 7200, overrideInClient, true, 2000);
try {
final String[] sessions = inComittedTransaction(session -> {
RealmModel realm = session.realms().getRealm(realmId);
UserModel user = session.users().getUserByUsername(realm, "user1");
UserSessionModel userSession = createUserSession(session, realm, user, offline);
Assert.assertEquals(offline, userSession.isOffline());
AuthenticatedClientSessionModel clientSession = createClientSession(session, realmId, realm.getClientByClientId("test-app"), userSession, "http://redirect", "state");
return new String[]{userSession.getId(), clientSession.getId()};
});
setTimeOffset(1000);
withRealm(realmId, (session, realm) -> {
// check the sessions are created
ClientModel client = realm.getClientByClientId("test-app");
UserSessionModel userSession = getUserSession(session, realm, sessions[0], offline);
Assert.assertNotNull(userSession);
Assert.assertNotNull(userSession.getAuthenticatedClientSessionByClient(client.getId()));
return null;
});
setTimeOffset(2100);
sessions[1] = withRealm(realmId, (session, realm) -> {
// refresh sessions after 2000 => only user session should exist
session.getContext().setRealm(realm);
ClientModel client = realm.getClientByClientId("test-app");
UserSessionModel userSession = getUserSession(session, realm, sessions[0], offline);
Assert.assertNotNull(userSession);
Assert.assertNull(userSession.getAuthenticatedClientSessionByClient(client.getId()));
// recreate client session
AuthenticatedClientSessionModel clientSession = createClientSession(session, realmId, realm.getClientByClientId("test-app"), userSession, "http://redirect", "state");
return clientSession.getId();
});
setTimeOffset(2500);
withRealm(realmId, (session, realm) -> {
// check the sessions are created
ClientModel client = realm.getClientByClientId("test-app");
UserSessionModel userSession = getUserSession(session, realm, sessions[0], offline);
Assert.assertNotNull(userSession);
Assert.assertNotNull(userSession.getAuthenticatedClientSessionByClient(client.getId()));
return null;
});
setTimeOffset(3100);
withRealm(realmId, (session, realm) -> {
// ensure user session is expired after user session expiration
Assert.assertNull(getUserSession(session, realm, sessions[0], offline));
return null;
});
} finally {
setTimeOffset(0);
}
}
protected void testUserClientMaxLifespanGreaterThanSession(boolean offline, boolean overrideInClient) {
configureTimeouts(3000, 7200, overrideInClient, true, 5000);
try {
final String[] sessions = withRealm(realmId, (session, realm) -> {
UserModel user = session.users().getUserByUsername(realm, "user1");
UserSessionModel userSession = createUserSession(session, realm, user, offline);
Assert.assertEquals(offline, userSession.isOffline());
AuthenticatedClientSessionModel clientSession = createClientSession(session, realmId, realm.getClientByClientId("test-app"), userSession, "http://redirect", "state");
return new String[]{userSession.getId(), clientSession.getId()};
});
setTimeOffset(2000);
withRealm(realmId, (session, realm) -> {
// check the sessions are created
ClientModel client = realm.getClientByClientId("test-app");
UserSessionModel userSession = getUserSession(session, realm, sessions[0], offline);
Assert.assertNotNull(userSession);
Assert.assertNotNull(userSession.getAuthenticatedClientSessionByClient(client.getId()));
return null;
});
setTimeOffset(3100);
withRealm(realmId, (session, realm) -> {
// ensure user session is expired after user session expiration
Assert.assertNull(getUserSession(session, realm, sessions[0], offline));
return null;
});
} finally {
setTimeOffset(0);
}
}
protected void testUserClientIdleTimeoutSmallerThanSession(int refreshTimes, boolean offline, boolean overrideInClient) {
configureTimeouts(7200, 3000, overrideInClient, false, 2000);
try {
final String[] sessions = withRealm(realmId, (session, realm) -> {
UserModel user = session.users().getUserByUsername(realm, "user1");
UserSessionModel userSession = createUserSession(session, realm, user, offline);
Assert.assertEquals(offline, userSession.isOffline());
AuthenticatedClientSessionModel clientSession = createClientSession(session, realmId, realm.getClientByClientId("test-app"), userSession, "http://redirect", "state");
return new String[]{userSession.getId(), clientSession.getId()};
});
int offset = 0;
for (int i = 0; i < refreshTimes; i++) {
offset += 1500;
setTimeOffset(offset);
withRealm(realmId, (session, realm) -> {
// refresh sessions before user session expires => both session should exist
ClientModel client = realm.getClientByClientId("test-app");
UserSessionModel userSession = getUserSession(session, realm, sessions[0], offline);
Assert.assertNotNull(userSession);
AuthenticatedClientSessionModel clientSession = userSession.getAuthenticatedClientSessionByClient(client.getId());
Assert.assertNotNull(clientSession);
userSession.setLastSessionRefresh(Time.currentTime());
clientSession.setTimestamp(Time.currentTime());
return null;
});
}
offset += 2100;
setTimeOffset(offset);
sessions[1] = withRealm(realmId, (session, realm) -> {
// refresh sessions after 2000 => only user session should exist, client should be expired by idle
session.getContext().setRealm(realm);
ClientModel client = realm.getClientByClientId("test-app");
UserSessionModel userSession = getUserSession(session, realm, sessions[0], offline);
Assert.assertNotNull(userSession);
Assert.assertNull(userSession.getAuthenticatedClientSessionByClient(client.getId()));
// recreate client session
AuthenticatedClientSessionModel clientSession = createClientSession(session, realmId, realm.getClientByClientId("test-app"), userSession, "http://redirect", "state");
return clientSession.getId();
});
offset += 3100;
setTimeOffset(offset);
withRealm(realmId, (session, realm) -> {
// ensure user session is expired after user session expiration
Assert.assertNull(getUserSession(session, realm, sessions[0], offline));
return null;
});
} finally {
setTimeOffset(0);
}
}
@Test
public void testOfflineUserClientMaxLifespanGreaterThanSession() {
testUserClientMaxLifespanGreaterThanSession(true, false);
}
@Test
public void testOfflineUserClientMaxLifespanGreaterThanSessionOverrideInClient() {
testUserClientMaxLifespanGreaterThanSession(true, true);
}
@Test
public void testOfflineUserClientMaxLifespanSmallerThanSession() {
testUserClientMaxLifespanSmallerThanSession(true, false);
}
@Test
public void testOfflineUserClientMaxLifespanSmallerThanSessionOverrideInClient() {
testUserClientMaxLifespanSmallerThanSession(true, true);
}
@Test
public void testOfflineUserClientIdleTimeoutSmallerThanSessionNoRefresh() {
testUserClientIdleTimeoutSmallerThanSession(0, true, false);
}
@Test
public void testOfflineUserClientIdleTimeoutSmallerThanSessionOneRefresh() {
testUserClientIdleTimeoutSmallerThanSession(1, true, false);
}
@Test
public void testOnlineUserClientMaxLifespanGreaterThanSession() {
testUserClientMaxLifespanGreaterThanSession(false, false);
}
@Test
public void testOnlineUserClientMaxLifespanGreaterThanSessionOverrideInClient() {
testUserClientMaxLifespanGreaterThanSession(false, true);
}
@Test
public void testOnlineUserClientMaxLifespanSmallerThanSession() {
testUserClientMaxLifespanSmallerThanSession(false, false);
}
@Test
public void testOnlineUserClientMaxLifespanSmallerThanSessionOverrideInClient() {
testUserClientMaxLifespanSmallerThanSession(false, true);
}
@Test
public void testOnlineUserClientIdleTimeoutSmallerThanSessionNoRefresh() {
testUserClientIdleTimeoutSmallerThanSession(0, false, false);
}
@Test
public void testOnlineUserClientIdleTimeoutSmallerThanSessionOneRefresh() {
testUserClientIdleTimeoutSmallerThanSession(1, false, false);
}
}

View file

@ -53,11 +53,11 @@ public class UserSessionExpirationTest extends KeycloakModelTest {
}
@Test
public void testClientSessionIdleTimeout() {
public void testSsoSessionIdleTimeout() {
// Set low ClientSessionIdleTimeout
// Set low ssoSessionIdleTimeout
withRealm(realmId, (session, realm) -> {
realm.setSsoSessionIdleTimeout(1800);
realm.setSsoSessionIdleTimeout(5);
realm.setSsoSessionMaxLifespan(36000);
realm.setClientSessionIdleTimeout(5);
return null;