mirror of
https://github.com/danny-avila/LibreChat.git
synced 2025-12-21 02:40:14 +01:00
🔄 refactor: MCP Registry System with Distributed Caching (#10191)
Some checks failed
Docker Dev Branch Images Build / build (Dockerfile, lc-dev, node) (push) Has been cancelled
Docker Dev Branch Images Build / build (Dockerfile.multi, lc-dev-api, api-build) (push) Has been cancelled
Docker Dev Images Build / build (Dockerfile, librechat-dev, node) (push) Has been cancelled
Docker Dev Images Build / build (Dockerfile.multi, librechat-dev-api, api-build) (push) Has been cancelled
Sync Locize Translations & Create Translation PR / Sync Translation Keys with Locize (push) Has been cancelled
Sync Locize Translations & Create Translation PR / Create Translation PR on Version Published (push) Has been cancelled
Some checks failed
Docker Dev Branch Images Build / build (Dockerfile, lc-dev, node) (push) Has been cancelled
Docker Dev Branch Images Build / build (Dockerfile.multi, lc-dev-api, api-build) (push) Has been cancelled
Docker Dev Images Build / build (Dockerfile, librechat-dev, node) (push) Has been cancelled
Docker Dev Images Build / build (Dockerfile.multi, librechat-dev-api, api-build) (push) Has been cancelled
Sync Locize Translations & Create Translation PR / Sync Translation Keys with Locize (push) Has been cancelled
Sync Locize Translations & Create Translation PR / Create Translation PR on Version Published (push) Has been cancelled
* refactor: Restructure MCP registry system with caching - Split MCPServersRegistry into modular components: - MCPServerInspector: handles server inspection and health checks - MCPServersInitializer: manages server initialization logic - MCPServersRegistry: simplified registry coordination - Add distributed caching layer: - ServerConfigsCacheRedis: Redis-backed configuration cache - ServerConfigsCacheInMemory: in-memory fallback cache - RegistryStatusCache: distributed leader election state - Add promise utilities (withTimeout) replacing Promise.race patterns - Add comprehensive cache integration tests for all cache implementations - Remove unused MCPManager.getAllToolFunctions method * fix: Update OAuth flow to include user-specific headers * chore: Update Jest configuration to ignore additional test files - Added patterns to ignore files ending with .helper.ts and .helper.d.ts in testPathIgnorePatterns for cleaner test runs. * fix: oauth headers in callback * chore: Update Jest testPathIgnorePatterns to exclude helper files - Modified testPathIgnorePatterns in package.json to ignore files ending with .helper.ts and .helper.d.ts for cleaner test execution. * ci: update test mocks --------- Co-authored-by: Danny Avila <danny@librechat.ai>
This commit is contained in:
parent
961f87cfda
commit
ce7e6edad8
45 changed files with 3116 additions and 1150 deletions
26
packages/api/src/mcp/registry/cache/BaseRegistryCache.ts
vendored
Normal file
26
packages/api/src/mcp/registry/cache/BaseRegistryCache.ts
vendored
Normal file
|
|
@ -0,0 +1,26 @@
|
|||
import type Keyv from 'keyv';
|
||||
import { isLeader } from '~/cluster';
|
||||
|
||||
/**
|
||||
* Base class for MCP registry caches that require distributed leader coordination.
|
||||
* Provides helper methods for leader-only operations and success validation.
|
||||
* All concrete implementations must provide their own Keyv cache instance.
|
||||
*/
|
||||
export abstract class BaseRegistryCache {
|
||||
protected readonly PREFIX = 'MCP::ServersRegistry';
|
||||
protected abstract readonly cache: Keyv;
|
||||
|
||||
protected async leaderCheck(action: string): Promise<void> {
|
||||
if (!(await isLeader())) throw new Error(`Only leader can ${action}.`);
|
||||
}
|
||||
|
||||
protected successCheck(action: string, success: boolean): true {
|
||||
if (!success) throw new Error(`Failed to ${action} in cache.`);
|
||||
return true;
|
||||
}
|
||||
|
||||
public async reset(): Promise<void> {
|
||||
await this.leaderCheck(`reset ${this.cache.namespace} cache`);
|
||||
await this.cache.clear();
|
||||
}
|
||||
}
|
||||
37
packages/api/src/mcp/registry/cache/RegistryStatusCache.ts
vendored
Normal file
37
packages/api/src/mcp/registry/cache/RegistryStatusCache.ts
vendored
Normal file
|
|
@ -0,0 +1,37 @@
|
|||
import { standardCache } from '~/cache';
|
||||
import { BaseRegistryCache } from './BaseRegistryCache';
|
||||
|
||||
// Status keys
|
||||
const INITIALIZED = 'INITIALIZED';
|
||||
|
||||
/**
|
||||
* Cache for tracking MCP Servers Registry metadata and status across distributed instances.
|
||||
* Uses Redis-backed storage to coordinate state between leader and follower nodes.
|
||||
* Currently, tracks initialization status to ensure only the leader performs initialization
|
||||
* while followers wait for completion. Designed to be extended with additional registry
|
||||
* metadata as needed (e.g., last update timestamps, version info, health status).
|
||||
* This cache is only meant to be used internally by registry management components.
|
||||
*/
|
||||
class RegistryStatusCache extends BaseRegistryCache {
|
||||
protected readonly cache = standardCache(`${this.PREFIX}::Status`);
|
||||
|
||||
public async isInitialized(): Promise<boolean> {
|
||||
return (await this.get(INITIALIZED)) === true;
|
||||
}
|
||||
|
||||
public async setInitialized(value: boolean): Promise<void> {
|
||||
await this.set(INITIALIZED, value);
|
||||
}
|
||||
|
||||
private async get<T = unknown>(key: string): Promise<T | undefined> {
|
||||
return this.cache.get(key);
|
||||
}
|
||||
|
||||
private async set(key: string, value: string | number | boolean, ttl?: number): Promise<void> {
|
||||
await this.leaderCheck('set MCP Servers Registry status');
|
||||
const success = await this.cache.set(key, value, ttl);
|
||||
this.successCheck(`set status key "${key}"`, success);
|
||||
}
|
||||
}
|
||||
|
||||
export const registryStatusCache = new RegistryStatusCache();
|
||||
31
packages/api/src/mcp/registry/cache/ServerConfigsCacheFactory.ts
vendored
Normal file
31
packages/api/src/mcp/registry/cache/ServerConfigsCacheFactory.ts
vendored
Normal file
|
|
@ -0,0 +1,31 @@
|
|||
import { cacheConfig } from '~/cache';
|
||||
import { ServerConfigsCacheInMemory } from './ServerConfigsCacheInMemory';
|
||||
import { ServerConfigsCacheRedis } from './ServerConfigsCacheRedis';
|
||||
|
||||
export type ServerConfigsCache = ServerConfigsCacheInMemory | ServerConfigsCacheRedis;
|
||||
|
||||
/**
|
||||
* Factory for creating the appropriate ServerConfigsCache implementation based on deployment mode.
|
||||
* Automatically selects between in-memory and Redis-backed storage depending on USE_REDIS config.
|
||||
* In single-instance mode (USE_REDIS=false), returns lightweight in-memory cache.
|
||||
* In cluster mode (USE_REDIS=true), returns Redis-backed cache with distributed coordination.
|
||||
* Provides a unified interface regardless of the underlying storage mechanism.
|
||||
*/
|
||||
export class ServerConfigsCacheFactory {
|
||||
/**
|
||||
* Create a ServerConfigsCache instance.
|
||||
* Returns Redis implementation if Redis is configured, otherwise in-memory implementation.
|
||||
*
|
||||
* @param owner - The owner of the cache (e.g., 'user', 'global') - only used for Redis namespacing
|
||||
* @param leaderOnly - Whether operations should only be performed by the leader (only applies to Redis)
|
||||
* @returns ServerConfigsCache instance
|
||||
*/
|
||||
static create(owner: string, leaderOnly: boolean): ServerConfigsCache {
|
||||
if (cacheConfig.USE_REDIS) {
|
||||
return new ServerConfigsCacheRedis(owner, leaderOnly);
|
||||
}
|
||||
|
||||
// In-memory mode uses a simple Map - doesn't need owner/namespace
|
||||
return new ServerConfigsCacheInMemory();
|
||||
}
|
||||
}
|
||||
46
packages/api/src/mcp/registry/cache/ServerConfigsCacheInMemory.ts
vendored
Normal file
46
packages/api/src/mcp/registry/cache/ServerConfigsCacheInMemory.ts
vendored
Normal file
|
|
@ -0,0 +1,46 @@
|
|||
import { ParsedServerConfig } from '~/mcp/types';
|
||||
|
||||
/**
|
||||
* In-memory implementation of MCP server configurations cache for single-instance deployments.
|
||||
* Uses a native JavaScript Map for fast, local storage without Redis dependencies.
|
||||
* Suitable for development environments or single-server production deployments.
|
||||
* Does not require leader checks or distributed coordination since data is instance-local.
|
||||
* Data is lost on server restart and not shared across multiple server instances.
|
||||
*/
|
||||
export class ServerConfigsCacheInMemory {
|
||||
private readonly cache: Map<string, ParsedServerConfig> = new Map();
|
||||
|
||||
public async add(serverName: string, config: ParsedServerConfig): Promise<void> {
|
||||
if (this.cache.has(serverName))
|
||||
throw new Error(
|
||||
`Server "${serverName}" already exists in cache. Use update() to modify existing configs.`,
|
||||
);
|
||||
this.cache.set(serverName, config);
|
||||
}
|
||||
|
||||
public async update(serverName: string, config: ParsedServerConfig): Promise<void> {
|
||||
if (!this.cache.has(serverName))
|
||||
throw new Error(
|
||||
`Server "${serverName}" does not exist in cache. Use add() to create new configs.`,
|
||||
);
|
||||
this.cache.set(serverName, config);
|
||||
}
|
||||
|
||||
public async remove(serverName: string): Promise<void> {
|
||||
if (!this.cache.delete(serverName)) {
|
||||
throw new Error(`Failed to remove server "${serverName}" in cache.`);
|
||||
}
|
||||
}
|
||||
|
||||
public async get(serverName: string): Promise<ParsedServerConfig | undefined> {
|
||||
return this.cache.get(serverName);
|
||||
}
|
||||
|
||||
public async getAll(): Promise<Record<string, ParsedServerConfig>> {
|
||||
return Object.fromEntries(this.cache);
|
||||
}
|
||||
|
||||
public async reset(): Promise<void> {
|
||||
this.cache.clear();
|
||||
}
|
||||
}
|
||||
80
packages/api/src/mcp/registry/cache/ServerConfigsCacheRedis.ts
vendored
Normal file
80
packages/api/src/mcp/registry/cache/ServerConfigsCacheRedis.ts
vendored
Normal file
|
|
@ -0,0 +1,80 @@
|
|||
import type Keyv from 'keyv';
|
||||
import { fromPairs } from 'lodash';
|
||||
import { standardCache, keyvRedisClient } from '~/cache';
|
||||
import { ParsedServerConfig } from '~/mcp/types';
|
||||
import { BaseRegistryCache } from './BaseRegistryCache';
|
||||
|
||||
/**
|
||||
* Redis-backed implementation of MCP server configurations cache for distributed deployments.
|
||||
* Stores server configs in Redis with namespace isolation by owner (App, User, or specific user ID).
|
||||
* Enables data sharing across multiple server instances in a cluster environment.
|
||||
* Supports optional leader-only write operations to prevent race conditions during initialization.
|
||||
* Data persists across server restarts and is accessible from any instance in the cluster.
|
||||
*/
|
||||
export class ServerConfigsCacheRedis extends BaseRegistryCache {
|
||||
protected readonly cache: Keyv;
|
||||
private readonly owner: string;
|
||||
private readonly leaderOnly: boolean;
|
||||
|
||||
constructor(owner: string, leaderOnly: boolean) {
|
||||
super();
|
||||
this.owner = owner;
|
||||
this.leaderOnly = leaderOnly;
|
||||
this.cache = standardCache(`${this.PREFIX}::Servers::${owner}`);
|
||||
}
|
||||
|
||||
public async add(serverName: string, config: ParsedServerConfig): Promise<void> {
|
||||
if (this.leaderOnly) await this.leaderCheck(`add ${this.owner} MCP servers`);
|
||||
const exists = await this.cache.has(serverName);
|
||||
if (exists)
|
||||
throw new Error(
|
||||
`Server "${serverName}" already exists in cache. Use update() to modify existing configs.`,
|
||||
);
|
||||
const success = await this.cache.set(serverName, config);
|
||||
this.successCheck(`add ${this.owner} server "${serverName}"`, success);
|
||||
}
|
||||
|
||||
public async update(serverName: string, config: ParsedServerConfig): Promise<void> {
|
||||
if (this.leaderOnly) await this.leaderCheck(`update ${this.owner} MCP servers`);
|
||||
const exists = await this.cache.has(serverName);
|
||||
if (!exists)
|
||||
throw new Error(
|
||||
`Server "${serverName}" does not exist in cache. Use add() to create new configs.`,
|
||||
);
|
||||
const success = await this.cache.set(serverName, config);
|
||||
this.successCheck(`update ${this.owner} server "${serverName}"`, success);
|
||||
}
|
||||
|
||||
public async remove(serverName: string): Promise<void> {
|
||||
if (this.leaderOnly) await this.leaderCheck(`remove ${this.owner} MCP servers`);
|
||||
const success = await this.cache.delete(serverName);
|
||||
this.successCheck(`remove ${this.owner} server "${serverName}"`, success);
|
||||
}
|
||||
|
||||
public async get(serverName: string): Promise<ParsedServerConfig | undefined> {
|
||||
return this.cache.get(serverName);
|
||||
}
|
||||
|
||||
public async getAll(): Promise<Record<string, ParsedServerConfig>> {
|
||||
// Use Redis SCAN iterator directly (non-blocking, production-ready)
|
||||
// Note: Keyv uses a single colon ':' between namespace and key, even if GLOBAL_PREFIX_SEPARATOR is '::'
|
||||
const pattern = `*${this.cache.namespace}:*`;
|
||||
const entries: Array<[string, ParsedServerConfig]> = [];
|
||||
|
||||
// Use scanIterator from Redis client
|
||||
if (keyvRedisClient && 'scanIterator' in keyvRedisClient) {
|
||||
for await (const key of keyvRedisClient.scanIterator({ MATCH: pattern })) {
|
||||
// Extract the actual key name (last part after final colon)
|
||||
// Full key format: "prefix::namespace:keyName"
|
||||
const lastColonIndex = key.lastIndexOf(':');
|
||||
const keyName = key.substring(lastColonIndex + 1);
|
||||
const value = await this.cache.get(keyName);
|
||||
if (value) {
|
||||
entries.push([keyName, value as ParsedServerConfig]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return fromPairs(entries);
|
||||
}
|
||||
}
|
||||
73
packages/api/src/mcp/registry/cache/__tests__/RegistryStatusCache.cache_integration.spec.ts
vendored
Normal file
73
packages/api/src/mcp/registry/cache/__tests__/RegistryStatusCache.cache_integration.spec.ts
vendored
Normal file
|
|
@ -0,0 +1,73 @@
|
|||
import { expect } from '@playwright/test';
|
||||
|
||||
describe('RegistryStatusCache Integration Tests', () => {
|
||||
let registryStatusCache: typeof import('../RegistryStatusCache').registryStatusCache;
|
||||
let keyvRedisClient: Awaited<typeof import('~/cache/redisClients')>['keyvRedisClient'];
|
||||
let LeaderElection: typeof import('~/cluster/LeaderElection').LeaderElection;
|
||||
let leaderInstance: InstanceType<typeof import('~/cluster/LeaderElection').LeaderElection>;
|
||||
|
||||
beforeAll(async () => {
|
||||
// Set up environment variables for Redis (only if not already set)
|
||||
process.env.USE_REDIS = process.env.USE_REDIS ?? 'true';
|
||||
process.env.REDIS_URI = process.env.REDIS_URI ?? 'redis://127.0.0.1:6379';
|
||||
process.env.REDIS_KEY_PREFIX =
|
||||
process.env.REDIS_KEY_PREFIX ?? 'RegistryStatusCache-IntegrationTest';
|
||||
|
||||
// Import modules after setting env vars
|
||||
const statusCacheModule = await import('../RegistryStatusCache');
|
||||
const redisClients = await import('~/cache/redisClients');
|
||||
const leaderElectionModule = await import('~/cluster/LeaderElection');
|
||||
|
||||
registryStatusCache = statusCacheModule.registryStatusCache;
|
||||
keyvRedisClient = redisClients.keyvRedisClient;
|
||||
LeaderElection = leaderElectionModule.LeaderElection;
|
||||
|
||||
// Ensure Redis is connected
|
||||
if (!keyvRedisClient) throw new Error('Redis client is not initialized');
|
||||
|
||||
// Wait for Redis to be ready
|
||||
if (!keyvRedisClient.isOpen) await keyvRedisClient.connect();
|
||||
|
||||
// Become leader so we can perform write operations
|
||||
leaderInstance = new LeaderElection();
|
||||
const isLeader = await leaderInstance.isLeader();
|
||||
expect(isLeader).toBe(true);
|
||||
});
|
||||
|
||||
afterEach(async () => {
|
||||
// Clean up: clear all test keys from Redis
|
||||
if (keyvRedisClient) {
|
||||
const pattern = '*RegistryStatusCache-IntegrationTest*';
|
||||
if ('scanIterator' in keyvRedisClient) {
|
||||
for await (const key of keyvRedisClient.scanIterator({ MATCH: pattern })) {
|
||||
await keyvRedisClient.del(key);
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
afterAll(async () => {
|
||||
// Resign as leader
|
||||
if (leaderInstance) await leaderInstance.resign();
|
||||
|
||||
// Close Redis connection
|
||||
if (keyvRedisClient?.isOpen) await keyvRedisClient.disconnect();
|
||||
});
|
||||
|
||||
describe('Initialization status tracking', () => {
|
||||
it('should return false for isInitialized when not set', async () => {
|
||||
const initialized = await registryStatusCache.isInitialized();
|
||||
expect(initialized).toBe(false);
|
||||
});
|
||||
|
||||
it('should set and get initialized status', async () => {
|
||||
await registryStatusCache.setInitialized(true);
|
||||
const initialized = await registryStatusCache.isInitialized();
|
||||
expect(initialized).toBe(true);
|
||||
|
||||
await registryStatusCache.setInitialized(false);
|
||||
const uninitialized = await registryStatusCache.isInitialized();
|
||||
expect(uninitialized).toBe(false);
|
||||
});
|
||||
});
|
||||
});
|
||||
70
packages/api/src/mcp/registry/cache/__tests__/ServerConfigsCacheFactory.test.ts
vendored
Normal file
70
packages/api/src/mcp/registry/cache/__tests__/ServerConfigsCacheFactory.test.ts
vendored
Normal file
|
|
@ -0,0 +1,70 @@
|
|||
import { ServerConfigsCacheFactory } from '../ServerConfigsCacheFactory';
|
||||
import { ServerConfigsCacheInMemory } from '../ServerConfigsCacheInMemory';
|
||||
import { ServerConfigsCacheRedis } from '../ServerConfigsCacheRedis';
|
||||
import { cacheConfig } from '~/cache';
|
||||
|
||||
// Mock the cache implementations
|
||||
jest.mock('../ServerConfigsCacheInMemory');
|
||||
jest.mock('../ServerConfigsCacheRedis');
|
||||
|
||||
// Mock the cache config module
|
||||
jest.mock('~/cache', () => ({
|
||||
cacheConfig: {
|
||||
USE_REDIS: false,
|
||||
},
|
||||
}));
|
||||
|
||||
describe('ServerConfigsCacheFactory', () => {
|
||||
beforeEach(() => {
|
||||
jest.clearAllMocks();
|
||||
});
|
||||
|
||||
describe('create()', () => {
|
||||
it('should return ServerConfigsCacheRedis when USE_REDIS is true', () => {
|
||||
// Arrange
|
||||
cacheConfig.USE_REDIS = true;
|
||||
|
||||
// Act
|
||||
const cache = ServerConfigsCacheFactory.create('TestOwner', true);
|
||||
|
||||
// Assert
|
||||
expect(cache).toBeInstanceOf(ServerConfigsCacheRedis);
|
||||
expect(ServerConfigsCacheRedis).toHaveBeenCalledWith('TestOwner', true);
|
||||
});
|
||||
|
||||
it('should return ServerConfigsCacheInMemory when USE_REDIS is false', () => {
|
||||
// Arrange
|
||||
cacheConfig.USE_REDIS = false;
|
||||
|
||||
// Act
|
||||
const cache = ServerConfigsCacheFactory.create('TestOwner', false);
|
||||
|
||||
// Assert
|
||||
expect(cache).toBeInstanceOf(ServerConfigsCacheInMemory);
|
||||
expect(ServerConfigsCacheInMemory).toHaveBeenCalled();
|
||||
});
|
||||
|
||||
it('should pass correct parameters to ServerConfigsCacheRedis', () => {
|
||||
// Arrange
|
||||
cacheConfig.USE_REDIS = true;
|
||||
|
||||
// Act
|
||||
ServerConfigsCacheFactory.create('App', true);
|
||||
|
||||
// Assert
|
||||
expect(ServerConfigsCacheRedis).toHaveBeenCalledWith('App', true);
|
||||
});
|
||||
|
||||
it('should create ServerConfigsCacheInMemory without parameters when USE_REDIS is false', () => {
|
||||
// Arrange
|
||||
cacheConfig.USE_REDIS = false;
|
||||
|
||||
// Act
|
||||
ServerConfigsCacheFactory.create('User', false);
|
||||
|
||||
// Assert
|
||||
// In-memory cache doesn't use owner/leaderOnly parameters
|
||||
expect(ServerConfigsCacheInMemory).toHaveBeenCalledWith();
|
||||
});
|
||||
});
|
||||
});
|
||||
173
packages/api/src/mcp/registry/cache/__tests__/ServerConfigsCacheInMemory.test.ts
vendored
Normal file
173
packages/api/src/mcp/registry/cache/__tests__/ServerConfigsCacheInMemory.test.ts
vendored
Normal file
|
|
@ -0,0 +1,173 @@
|
|||
import { expect } from '@playwright/test';
|
||||
import { ParsedServerConfig } from '~/mcp/types';
|
||||
|
||||
describe('ServerConfigsCacheInMemory Integration Tests', () => {
|
||||
let ServerConfigsCacheInMemory: typeof import('../ServerConfigsCacheInMemory').ServerConfigsCacheInMemory;
|
||||
let cache: InstanceType<
|
||||
typeof import('../ServerConfigsCacheInMemory').ServerConfigsCacheInMemory
|
||||
>;
|
||||
|
||||
// Test data
|
||||
const mockConfig1: ParsedServerConfig = {
|
||||
command: 'node',
|
||||
args: ['server1.js'],
|
||||
env: { TEST: 'value1' },
|
||||
};
|
||||
|
||||
const mockConfig2: ParsedServerConfig = {
|
||||
command: 'python',
|
||||
args: ['server2.py'],
|
||||
env: { TEST: 'value2' },
|
||||
};
|
||||
|
||||
const mockConfig3: ParsedServerConfig = {
|
||||
command: 'node',
|
||||
args: ['server3.js'],
|
||||
url: 'http://localhost:3000',
|
||||
requiresOAuth: true,
|
||||
};
|
||||
|
||||
beforeAll(async () => {
|
||||
// Import modules
|
||||
const cacheModule = await import('../ServerConfigsCacheInMemory');
|
||||
ServerConfigsCacheInMemory = cacheModule.ServerConfigsCacheInMemory;
|
||||
});
|
||||
|
||||
beforeEach(() => {
|
||||
// Create a fresh instance for each test
|
||||
cache = new ServerConfigsCacheInMemory();
|
||||
});
|
||||
|
||||
describe('add and get operations', () => {
|
||||
it('should add and retrieve a server config', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
const result = await cache.get('server1');
|
||||
expect(result).toEqual(mockConfig1);
|
||||
});
|
||||
|
||||
it('should return undefined for non-existent server', async () => {
|
||||
const result = await cache.get('non-existent');
|
||||
expect(result).toBeUndefined();
|
||||
});
|
||||
|
||||
it('should throw error when adding duplicate server', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
await expect(cache.add('server1', mockConfig2)).rejects.toThrow(
|
||||
'Server "server1" already exists in cache. Use update() to modify existing configs.',
|
||||
);
|
||||
});
|
||||
|
||||
it('should handle multiple server configs', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
await cache.add('server2', mockConfig2);
|
||||
await cache.add('server3', mockConfig3);
|
||||
|
||||
const result1 = await cache.get('server1');
|
||||
const result2 = await cache.get('server2');
|
||||
const result3 = await cache.get('server3');
|
||||
|
||||
expect(result1).toEqual(mockConfig1);
|
||||
expect(result2).toEqual(mockConfig2);
|
||||
expect(result3).toEqual(mockConfig3);
|
||||
});
|
||||
});
|
||||
|
||||
describe('getAll operation', () => {
|
||||
it('should return empty object when no servers exist', async () => {
|
||||
const result = await cache.getAll();
|
||||
expect(result).toEqual({});
|
||||
});
|
||||
|
||||
it('should return all server configs', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
await cache.add('server2', mockConfig2);
|
||||
await cache.add('server3', mockConfig3);
|
||||
|
||||
const result = await cache.getAll();
|
||||
expect(result).toEqual({
|
||||
server1: mockConfig1,
|
||||
server2: mockConfig2,
|
||||
server3: mockConfig3,
|
||||
});
|
||||
});
|
||||
|
||||
it('should reflect updates in getAll', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
await cache.add('server2', mockConfig2);
|
||||
|
||||
let result = await cache.getAll();
|
||||
expect(Object.keys(result).length).toBe(2);
|
||||
|
||||
await cache.add('server3', mockConfig3);
|
||||
result = await cache.getAll();
|
||||
expect(Object.keys(result).length).toBe(3);
|
||||
expect(result.server3).toEqual(mockConfig3);
|
||||
});
|
||||
});
|
||||
|
||||
describe('update operation', () => {
|
||||
it('should update an existing server config', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
expect(await cache.get('server1')).toEqual(mockConfig1);
|
||||
|
||||
await cache.update('server1', mockConfig2);
|
||||
const result = await cache.get('server1');
|
||||
expect(result).toEqual(mockConfig2);
|
||||
});
|
||||
|
||||
it('should throw error when updating non-existent server', async () => {
|
||||
await expect(cache.update('non-existent', mockConfig1)).rejects.toThrow(
|
||||
'Server "non-existent" does not exist in cache. Use add() to create new configs.',
|
||||
);
|
||||
});
|
||||
|
||||
it('should reflect updates in getAll', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
await cache.add('server2', mockConfig2);
|
||||
|
||||
await cache.update('server1', mockConfig3);
|
||||
const result = await cache.getAll();
|
||||
expect(result.server1).toEqual(mockConfig3);
|
||||
expect(result.server2).toEqual(mockConfig2);
|
||||
});
|
||||
});
|
||||
|
||||
describe('remove operation', () => {
|
||||
it('should remove an existing server config', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
expect(await cache.get('server1')).toEqual(mockConfig1);
|
||||
|
||||
await cache.remove('server1');
|
||||
expect(await cache.get('server1')).toBeUndefined();
|
||||
});
|
||||
|
||||
it('should throw error when removing non-existent server', async () => {
|
||||
await expect(cache.remove('non-existent')).rejects.toThrow(
|
||||
'Failed to remove server "non-existent" in cache.',
|
||||
);
|
||||
});
|
||||
|
||||
it('should remove server from getAll results', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
await cache.add('server2', mockConfig2);
|
||||
|
||||
let result = await cache.getAll();
|
||||
expect(Object.keys(result).length).toBe(2);
|
||||
|
||||
await cache.remove('server1');
|
||||
result = await cache.getAll();
|
||||
expect(Object.keys(result).length).toBe(1);
|
||||
expect(result.server1).toBeUndefined();
|
||||
expect(result.server2).toEqual(mockConfig2);
|
||||
});
|
||||
|
||||
it('should allow re-adding a removed server', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
await cache.remove('server1');
|
||||
await cache.add('server1', mockConfig3);
|
||||
|
||||
const result = await cache.get('server1');
|
||||
expect(result).toEqual(mockConfig3);
|
||||
});
|
||||
});
|
||||
});
|
||||
278
packages/api/src/mcp/registry/cache/__tests__/ServerConfigsCacheRedis.cache_integration.spec.ts
vendored
Normal file
278
packages/api/src/mcp/registry/cache/__tests__/ServerConfigsCacheRedis.cache_integration.spec.ts
vendored
Normal file
|
|
@ -0,0 +1,278 @@
|
|||
import { expect } from '@playwright/test';
|
||||
import { ParsedServerConfig } from '~/mcp/types';
|
||||
|
||||
describe('ServerConfigsCacheRedis Integration Tests', () => {
|
||||
let ServerConfigsCacheRedis: typeof import('../ServerConfigsCacheRedis').ServerConfigsCacheRedis;
|
||||
let keyvRedisClient: Awaited<typeof import('~/cache/redisClients')>['keyvRedisClient'];
|
||||
let LeaderElection: typeof import('~/cluster/LeaderElection').LeaderElection;
|
||||
let checkIsLeader: () => Promise<boolean>;
|
||||
let cache: InstanceType<typeof import('../ServerConfigsCacheRedis').ServerConfigsCacheRedis>;
|
||||
|
||||
// Test data
|
||||
const mockConfig1: ParsedServerConfig = {
|
||||
command: 'node',
|
||||
args: ['server1.js'],
|
||||
env: { TEST: 'value1' },
|
||||
};
|
||||
|
||||
const mockConfig2: ParsedServerConfig = {
|
||||
command: 'python',
|
||||
args: ['server2.py'],
|
||||
env: { TEST: 'value2' },
|
||||
};
|
||||
|
||||
const mockConfig3: ParsedServerConfig = {
|
||||
command: 'node',
|
||||
args: ['server3.js'],
|
||||
url: 'http://localhost:3000',
|
||||
requiresOAuth: true,
|
||||
};
|
||||
|
||||
beforeAll(async () => {
|
||||
// Set up environment variables for Redis (only if not already set)
|
||||
process.env.USE_REDIS = process.env.USE_REDIS ?? 'true';
|
||||
process.env.REDIS_URI = process.env.REDIS_URI ?? 'redis://127.0.0.1:6379';
|
||||
process.env.REDIS_KEY_PREFIX =
|
||||
process.env.REDIS_KEY_PREFIX ?? 'ServerConfigsCacheRedis-IntegrationTest';
|
||||
|
||||
// Import modules after setting env vars
|
||||
const cacheModule = await import('../ServerConfigsCacheRedis');
|
||||
const redisClients = await import('~/cache/redisClients');
|
||||
const leaderElectionModule = await import('~/cluster/LeaderElection');
|
||||
const clusterModule = await import('~/cluster');
|
||||
|
||||
ServerConfigsCacheRedis = cacheModule.ServerConfigsCacheRedis;
|
||||
keyvRedisClient = redisClients.keyvRedisClient;
|
||||
LeaderElection = leaderElectionModule.LeaderElection;
|
||||
checkIsLeader = clusterModule.isLeader;
|
||||
|
||||
// Ensure Redis is connected
|
||||
if (!keyvRedisClient) throw new Error('Redis client is not initialized');
|
||||
|
||||
// Wait for Redis to be ready
|
||||
if (!keyvRedisClient.isOpen) await keyvRedisClient.connect();
|
||||
|
||||
// Clear any existing leader key to ensure clean state
|
||||
await keyvRedisClient.del(LeaderElection.LEADER_KEY);
|
||||
|
||||
// Become leader so we can perform write operations (using default election instance)
|
||||
const isLeader = await checkIsLeader();
|
||||
expect(isLeader).toBe(true);
|
||||
});
|
||||
|
||||
beforeEach(() => {
|
||||
// Create a fresh instance for each test with leaderOnly=true
|
||||
cache = new ServerConfigsCacheRedis('test-user', true);
|
||||
});
|
||||
|
||||
afterEach(async () => {
|
||||
// Clean up: clear all test keys from Redis
|
||||
if (keyvRedisClient) {
|
||||
const pattern = '*ServerConfigsCacheRedis-IntegrationTest*';
|
||||
if ('scanIterator' in keyvRedisClient) {
|
||||
for await (const key of keyvRedisClient.scanIterator({ MATCH: pattern })) {
|
||||
await keyvRedisClient.del(key);
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
afterAll(async () => {
|
||||
// Clear leader key to allow other tests to become leader
|
||||
if (keyvRedisClient) await keyvRedisClient.del(LeaderElection.LEADER_KEY);
|
||||
|
||||
// Close Redis connection
|
||||
if (keyvRedisClient?.isOpen) await keyvRedisClient.disconnect();
|
||||
});
|
||||
|
||||
describe('add and get operations', () => {
|
||||
it('should add and retrieve a server config', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
const result = await cache.get('server1');
|
||||
expect(result).toEqual(mockConfig1);
|
||||
});
|
||||
|
||||
it('should return undefined for non-existent server', async () => {
|
||||
const result = await cache.get('non-existent');
|
||||
expect(result).toBeUndefined();
|
||||
});
|
||||
|
||||
it('should throw error when adding duplicate server', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
await expect(cache.add('server1', mockConfig2)).rejects.toThrow(
|
||||
'Server "server1" already exists in cache. Use update() to modify existing configs.',
|
||||
);
|
||||
});
|
||||
|
||||
it('should handle multiple server configs', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
await cache.add('server2', mockConfig2);
|
||||
await cache.add('server3', mockConfig3);
|
||||
|
||||
const result1 = await cache.get('server1');
|
||||
const result2 = await cache.get('server2');
|
||||
const result3 = await cache.get('server3');
|
||||
|
||||
expect(result1).toEqual(mockConfig1);
|
||||
expect(result2).toEqual(mockConfig2);
|
||||
expect(result3).toEqual(mockConfig3);
|
||||
});
|
||||
|
||||
it('should isolate caches by owner namespace', async () => {
|
||||
const userCache = new ServerConfigsCacheRedis('user1', true);
|
||||
const globalCache = new ServerConfigsCacheRedis('global', true);
|
||||
|
||||
await userCache.add('server1', mockConfig1);
|
||||
await globalCache.add('server1', mockConfig2);
|
||||
|
||||
const userResult = await userCache.get('server1');
|
||||
const globalResult = await globalCache.get('server1');
|
||||
|
||||
expect(userResult).toEqual(mockConfig1);
|
||||
expect(globalResult).toEqual(mockConfig2);
|
||||
});
|
||||
});
|
||||
|
||||
describe('getAll operation', () => {
|
||||
it('should return empty object when no servers exist', async () => {
|
||||
const result = await cache.getAll();
|
||||
expect(result).toEqual({});
|
||||
});
|
||||
|
||||
it('should return all server configs', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
await cache.add('server2', mockConfig2);
|
||||
await cache.add('server3', mockConfig3);
|
||||
|
||||
const result = await cache.getAll();
|
||||
expect(result).toEqual({
|
||||
server1: mockConfig1,
|
||||
server2: mockConfig2,
|
||||
server3: mockConfig3,
|
||||
});
|
||||
});
|
||||
|
||||
it('should reflect updates in getAll', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
await cache.add('server2', mockConfig2);
|
||||
|
||||
let result = await cache.getAll();
|
||||
expect(Object.keys(result).length).toBe(2);
|
||||
|
||||
await cache.add('server3', mockConfig3);
|
||||
result = await cache.getAll();
|
||||
expect(Object.keys(result).length).toBe(3);
|
||||
expect(result.server3).toEqual(mockConfig3);
|
||||
});
|
||||
|
||||
it('should only return configs for the specific owner', async () => {
|
||||
const userCache = new ServerConfigsCacheRedis('user1', true);
|
||||
const globalCache = new ServerConfigsCacheRedis('global', true);
|
||||
|
||||
await userCache.add('server1', mockConfig1);
|
||||
await userCache.add('server2', mockConfig2);
|
||||
await globalCache.add('server3', mockConfig3);
|
||||
|
||||
const userResult = await userCache.getAll();
|
||||
const globalResult = await globalCache.getAll();
|
||||
|
||||
expect(Object.keys(userResult).length).toBe(2);
|
||||
expect(Object.keys(globalResult).length).toBe(1);
|
||||
expect(userResult.server1).toEqual(mockConfig1);
|
||||
expect(userResult.server3).toBeUndefined();
|
||||
expect(globalResult.server3).toEqual(mockConfig3);
|
||||
});
|
||||
});
|
||||
|
||||
describe('update operation', () => {
|
||||
it('should update an existing server config', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
expect(await cache.get('server1')).toEqual(mockConfig1);
|
||||
|
||||
await cache.update('server1', mockConfig2);
|
||||
const result = await cache.get('server1');
|
||||
expect(result).toEqual(mockConfig2);
|
||||
});
|
||||
|
||||
it('should throw error when updating non-existent server', async () => {
|
||||
await expect(cache.update('non-existent', mockConfig1)).rejects.toThrow(
|
||||
'Server "non-existent" does not exist in cache. Use add() to create new configs.',
|
||||
);
|
||||
});
|
||||
|
||||
it('should reflect updates in getAll', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
await cache.add('server2', mockConfig2);
|
||||
|
||||
await cache.update('server1', mockConfig3);
|
||||
const result = await cache.getAll();
|
||||
expect(result.server1).toEqual(mockConfig3);
|
||||
expect(result.server2).toEqual(mockConfig2);
|
||||
});
|
||||
|
||||
it('should only update in the specific owner namespace', async () => {
|
||||
const userCache = new ServerConfigsCacheRedis('user1', true);
|
||||
const globalCache = new ServerConfigsCacheRedis('global', true);
|
||||
|
||||
await userCache.add('server1', mockConfig1);
|
||||
await globalCache.add('server1', mockConfig2);
|
||||
|
||||
await userCache.update('server1', mockConfig3);
|
||||
|
||||
expect(await userCache.get('server1')).toEqual(mockConfig3);
|
||||
expect(await globalCache.get('server1')).toEqual(mockConfig2);
|
||||
});
|
||||
});
|
||||
|
||||
describe('remove operation', () => {
|
||||
it('should remove an existing server config', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
expect(await cache.get('server1')).toEqual(mockConfig1);
|
||||
|
||||
await cache.remove('server1');
|
||||
expect(await cache.get('server1')).toBeUndefined();
|
||||
});
|
||||
|
||||
it('should throw error when removing non-existent server', async () => {
|
||||
await expect(cache.remove('non-existent')).rejects.toThrow(
|
||||
'Failed to remove test-user server "non-existent"',
|
||||
);
|
||||
});
|
||||
|
||||
it('should remove server from getAll results', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
await cache.add('server2', mockConfig2);
|
||||
|
||||
let result = await cache.getAll();
|
||||
expect(Object.keys(result).length).toBe(2);
|
||||
|
||||
await cache.remove('server1');
|
||||
result = await cache.getAll();
|
||||
expect(Object.keys(result).length).toBe(1);
|
||||
expect(result.server1).toBeUndefined();
|
||||
expect(result.server2).toEqual(mockConfig2);
|
||||
});
|
||||
|
||||
it('should allow re-adding a removed server', async () => {
|
||||
await cache.add('server1', mockConfig1);
|
||||
await cache.remove('server1');
|
||||
await cache.add('server1', mockConfig3);
|
||||
|
||||
const result = await cache.get('server1');
|
||||
expect(result).toEqual(mockConfig3);
|
||||
});
|
||||
|
||||
it('should only remove from the specific owner namespace', async () => {
|
||||
const userCache = new ServerConfigsCacheRedis('user1', true);
|
||||
const globalCache = new ServerConfigsCacheRedis('global', true);
|
||||
|
||||
await userCache.add('server1', mockConfig1);
|
||||
await globalCache.add('server1', mockConfig2);
|
||||
|
||||
await userCache.remove('server1');
|
||||
|
||||
expect(await userCache.get('server1')).toBeUndefined();
|
||||
expect(await globalCache.get('server1')).toEqual(mockConfig2);
|
||||
});
|
||||
});
|
||||
});
|
||||
Loading…
Add table
Add a link
Reference in a new issue